diff --git a/.github/workflows/docker_scan.yml b/.github/workflows/docker_scan.yml new file mode 100644 index 0000000000..7d9ecfe619 --- /dev/null +++ b/.github/workflows/docker_scan.yml @@ -0,0 +1,44 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: Docker Image CVE Scanner +on: + schedule: + # This job will run at 3:30 UTC daily + - cron: '30 3 * * *' + workflow_dispatch: +jobs: + scan_jvm: + runs-on: ubuntu-latest + strategy: + matrix: + # This is an array of supported tags. Make sure this array only contains the supported tags + supported_image_tag: ['latest', '3.7.0'] + steps: + - name: Run CVE scan + uses: aquasecurity/trivy-action@master + if: always() + with: + image-ref: apache/kafka:${{ matrix.supported_image_tag }} + format: 'table' + severity: 'CRITICAL,HIGH' + output: scan_report_jvm_${{ matrix.supported_image_tag }}.txt + exit-code: '1' + - name: Upload CVE scan report + if: always() + uses: actions/upload-artifact@v3 + with: + name: scan_report_jvm_${{ matrix.supported_image_tag }}.txt + path: scan_report_jvm_${{ matrix.supported_image_tag }}.txt diff --git a/.gitignore b/.gitignore index 4ac36a815b..7dfe61c38c 100644 --- a/.gitignore +++ b/.gitignore @@ -34,7 +34,6 @@ Vagrantfile.local config/server-* config/zookeeper-* -core/data/* gradle/wrapper/*.jar gradlew.bat diff --git a/LICENSE-binary b/LICENSE-binary index 106ee8e7fa..a395f15e42 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -70,8 +70,8 @@ scala-java8-compat_2.12-1.0.2 scala-java8-compat_2.13-1.0.2 snappy-java-1.1.10.5 swagger-annotations-2.2.8 -zookeeper-3.8.3 -zookeeper-jute-3.8.3 +zookeeper-3.8.4 +zookeeper-jute-3.8.4 =============================================================================== This product bundles various third-party components under other open source @@ -133,7 +133,7 @@ zstd-jni-1.5.5-11 see: licenses/zstd-jni-BSD-2-clause --------------------------------------- BSD 3-Clause -jline-3.22.0, see: licenses/jline-BSD-3-clause +jline-3.25.1, see: licenses/jline-BSD-3-clause jsr305-3.0.2, see: licenses/jsr305-BSD-3-clause paranamer-2.8, see: licenses/paranamer-BSD-3-clause protobuf-java-3.23.4, see: licenses/protobuf-java-BSD-3-clause diff --git a/bin/kafka-consumer-groups.sh b/bin/kafka-consumer-groups.sh index feb063de75..6dde7d708f 100755 --- a/bin/kafka-consumer-groups.sh +++ b/bin/kafka-consumer-groups.sh @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -exec $(dirname $0)/kafka-run-class.sh kafka.admin.ConsumerGroupCommand "$@" +exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.tools.consumer.group.ConsumerGroupCommand "$@" diff --git a/bin/windows/kafka-consumer-groups.bat b/bin/windows/kafka-consumer-groups.bat index e027b9e6bf..bdec36be41 100644 --- a/bin/windows/kafka-consumer-groups.bat +++ b/bin/windows/kafka-consumer-groups.bat @@ -14,4 +14,4 @@ rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. rem See the License for the specific language governing permissions and rem limitations under the License. -"%~dp0kafka-run-class.bat" kafka.admin.ConsumerGroupCommand %* +"%~dp0kafka-run-class.bat" org.apache.kafka.tools.consumer.group.ConsumerGroupCommand %* diff --git a/build.gradle b/build.gradle index 0feecfd106..7f5f5d4aa6 100644 --- a/build.gradle +++ b/build.gradle @@ -43,7 +43,7 @@ plugins { // includes spotbugs version 4.7.4, in which case CVE-2022-42920 can // be dropped from gradle/resources/dependencycheck-suppressions.xml id "com.github.spotbugs" version '5.1.3' apply false - id 'org.scoverage' version '7.0.1' apply false + id 'org.scoverage' version '8.0.3' apply false // Updating the shadow plugin version to 8.1.1 causes issue with signing and publishing the shadowed // artifacts - see https://github.com/johnrengelman/shadow/issues/901 id 'com.github.johnrengelman.shadow' version '8.1.0' apply false @@ -380,8 +380,7 @@ subprojects { if (JavaVersion.current().isCompatibleWith(JavaVersion.VERSION_16)) { testsToExclude.addAll([ // connect tests - "**/KafkaConfigBackingStoreTest.*", - "**/WorkerSinkTaskTest.*" + "**/KafkaConfigBackingStoreTest.*" ]) } @@ -745,7 +744,7 @@ subprojects { if (userEnableTestCoverage) { def coverageGen = it.path == ':core' ? 'reportScoverage' : 'jacocoTestReport' - task reportCoverage(dependsOn: [coverageGen]) + tasks.register('reportCoverage').configure { dependsOn(coverageGen) } } dependencyCheck { @@ -795,33 +794,8 @@ def checkstyleConfigProperties(configFileName) { [importControlFile: "$configFileName"] } -// Aggregates all jacoco results into the root project directory if (userEnableTestCoverage) { - task jacocoRootReport(type: org.gradle.testing.jacoco.tasks.JacocoReport) { - def javaProjects = subprojects.findAll { it.path != ':core' } - - description = 'Generates an aggregate report from all subprojects' - dependsOn(javaProjects.test) - - additionalSourceDirs.from = javaProjects.sourceSets.main.allSource.srcDirs - sourceDirectories.from = javaProjects.sourceSets.main.allSource.srcDirs - classDirectories.from = javaProjects.sourceSets.main.output - executionData.from = javaProjects.jacocoTestReport.executionData - - reports { - html.required = true - xml.required = true - } - // workaround to ignore projects that don't have any tests at all - onlyIf = { true } - doFirst { - executionData = files(executionData.findAll { it.exists() }) - } - } -} - -if (userEnableTestCoverage) { - task reportCoverage(dependsOn: ['jacocoRootReport', 'core:reportCoverage']) + tasks.register('reportCoverage').configure { dependsOn(subprojects.reportCoverage) } } def connectPkgs = [ @@ -1027,6 +1001,9 @@ project(':core') { if (userEnableTestCoverage) { scoverage { scoverageVersion = versions.scoverage + if (versions.baseScala == '2.13') { + scoverageScalaVersion = '2.13.9' // there's no newer 2.13 artifact, org.scoverage:scalac-scoverage-plugin_2.13.9:2.0.11 is the latest as of now + } reportDir = file("${rootProject.buildDir}/scoverage") highlighting = false minimumRate = 0.0 @@ -1339,7 +1316,7 @@ project(':metadata') { } test { java { - srcDirs = ["src/generated/java", "src/test/java"] + srcDirs = ["src/test/java"] } } } @@ -1706,7 +1683,7 @@ project(':raft') { } test { java { - srcDirs = ["src/generated/java", "src/test/java"] + srcDirs = ["src/test/java"] } } } @@ -1951,7 +1928,7 @@ project(':storage') { } test { java { - srcDirs = ["src/generated/java", "src/test/java"] + srcDirs = ["src/test/java"] } } } @@ -2107,6 +2084,9 @@ project(':tools') { implementation project(':connect:runtime') implementation project(':tools:tools-api') implementation libs.argparse4j + implementation libs.jacksonDatabind + implementation libs.jacksonDataformatCsv + implementation libs.jacksonJDK8Datatypes implementation libs.slf4jApi implementation libs.log4j implementation libs.joptSimple @@ -2305,14 +2285,13 @@ project(':streams') { testImplementation project(':core') testImplementation project(':tools') testImplementation project(':core').sourceSets.test.output + testImplementation project(':storage') testImplementation project(':server-common') testImplementation project(':server-common').sourceSets.test.output testImplementation libs.log4j testImplementation libs.junitJupiter testImplementation libs.junitVintageEngine testImplementation libs.easymock - testImplementation libs.powermockJunit4 - testImplementation libs.powermockEasymock testImplementation libs.bcpkix testImplementation libs.hamcrest testImplementation libs.mockitoCore @@ -2347,7 +2326,7 @@ project(':streams') { } test { java { - srcDirs = ["src/generated/java", "src/test/java"] + srcDirs = ["src/test/java"] } } } @@ -2432,6 +2411,7 @@ project(':streams') { ':streams:upgrade-system-tests-34:test', ':streams:upgrade-system-tests-35:test', ':streams:upgrade-system-tests-36:test', + ':streams:upgrade-system-tests-37:test', ':streams:examples:test' ] ) @@ -2460,7 +2440,6 @@ project(':streams:streams-scala') { testImplementation project(':streams:test-utils') testImplementation libs.junitJupiter - testImplementation libs.easymock testImplementation libs.mockitoCore testImplementation libs.mockitoJunitJupiter // supports MockitoExtension testImplementation libs.hamcrest @@ -2851,7 +2830,6 @@ project(':streams:upgrade-system-tests-35') { } } - project(':streams:upgrade-system-tests-36') { archivesBaseName = "kafka-streams-upgrade-system-tests-36" @@ -2865,6 +2843,19 @@ project(':streams:upgrade-system-tests-36') { } } +project(':streams:upgrade-system-tests-37') { + archivesBaseName = "kafka-streams-upgrade-system-tests-37" + + dependencies { + testImplementation libs.kafkaStreams_37 + testRuntimeOnly libs.junitJupiter + } + + systemTestLibs { + dependsOn testJar + } +} + project(':jmh-benchmarks') { apply plugin: 'com.github.johnrengelman.shadow' @@ -2997,7 +2988,6 @@ project(':connect:transforms') { implementation libs.slf4jApi - testImplementation libs.easymock testImplementation libs.junitJupiter testRuntimeOnly libs.slf4jlog4j @@ -3037,8 +3027,7 @@ project(':connect:json') { api libs.jacksonAfterburner implementation libs.slf4jApi - - testImplementation libs.easymock + testImplementation libs.junitJupiter testRuntimeOnly libs.slf4jlog4j @@ -3108,6 +3097,7 @@ project(':connect:runtime') { testImplementation project(':metadata') testImplementation project(':core').sourceSets.test.output testImplementation project(':server-common') + testImplementation project(':storage') testImplementation project(':connect:test-plugins') testImplementation libs.easymock @@ -3179,7 +3169,7 @@ project(':connect:runtime') { } task genConnectMetricsDocs(type: JavaExec) { - classpath = sourceSets.test.runtimeClasspath + classpath = sourceSets.main.runtimeClasspath mainClass = 'org.apache.kafka.connect.runtime.ConnectMetrics' if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } standardOutput = new File(generatedDocsDir, "connect_metrics.html").newOutputStream() diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml index 3f9a21fffc..782d2fe461 100644 --- a/checkstyle/import-control-core.xml +++ b/checkstyle/import-control-core.xml @@ -56,6 +56,7 @@ + diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index caf1fe5ebe..8bbf572821 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -326,6 +326,8 @@ + + @@ -410,6 +412,7 @@ + @@ -601,6 +604,7 @@ + diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index bce1949c12..8df1a0d589 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -46,6 +46,7 @@ + + files="(AbstractFetch|ClientTelemetryReporter|ConsumerCoordinator|CommitRequestManager|FetchCollector|OffsetFetcherUtils|KafkaProducer|Sender|ConfigDef|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator|AbstractCoordinator|TransactionManager|AbstractStickyAssignor|DefaultSslEngineFactory|Authorizer|RecordAccumulator|MemoryRecords|FetchSessionHandler|MockAdminClient).java"/> @@ -172,12 +173,12 @@ files="(DistributedHerder|KafkaBasedLog|WorkerSourceTaskWithTopicCreation|WorkerSourceTask)Test.java"/> + files="(WorkerSink|WorkerSource|ErrorHandling)Task(|WithTopicCreation|Mockito)Test.java"/> + files="(RequestResponse|WorkerSinkTask|WorkerSinkTaskMockito)Test.java"/> @@ -233,7 +234,7 @@ + files="(RecordCollectorTest|StreamsPartitionAssignorTest|StreamThreadTest|StreamTaskTest|TaskManagerTest|TopologyTestDriverTest|KafkaStreamsTest|EosIntegrationTest|RestoreIntegrationTest).java"/> diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index 5180b9d726..dbd0124c00 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -1644,7 +1644,7 @@ default ListTransactionsResult listTransactions() { * should typically attempt to reduce the size of the result set using * {@link ListTransactionsOptions#filterProducerIds(Collection)} or * {@link ListTransactionsOptions#filterStates(Collection)} or - * {@link ListTransactionsOptions#durationFilter(Long)} + * {@link ListTransactionsOptions#filterOnDuration(long)}. * * @param options Options to control the method behavior (including filters) * @return The result diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ConsumerGroupListing.java b/clients/src/main/java/org/apache/kafka/clients/admin/ConsumerGroupListing.java index 0abc3e01ca..01c23796d4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ConsumerGroupListing.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ConsumerGroupListing.java @@ -21,6 +21,7 @@ import java.util.Optional; import org.apache.kafka.common.ConsumerGroupState; +import org.apache.kafka.common.GroupType; /** * A listing of a consumer group in the cluster. @@ -29,6 +30,7 @@ public class ConsumerGroupListing { private final String groupId; private final boolean isSimpleConsumerGroup; private final Optional state; + private final Optional type; /** * Create an instance with the specified parameters. @@ -37,7 +39,7 @@ public class ConsumerGroupListing { * @param isSimpleConsumerGroup If consumer group is simple or not. */ public ConsumerGroupListing(String groupId, boolean isSimpleConsumerGroup) { - this(groupId, isSimpleConsumerGroup, Optional.empty()); + this(groupId, isSimpleConsumerGroup, Optional.empty(), Optional.empty()); } /** @@ -48,9 +50,27 @@ public ConsumerGroupListing(String groupId, boolean isSimpleConsumerGroup) { * @param state The state of the consumer group */ public ConsumerGroupListing(String groupId, boolean isSimpleConsumerGroup, Optional state) { + this(groupId, isSimpleConsumerGroup, state, Optional.empty()); + } + + /** + * Create an instance with the specified parameters. + * + * @param groupId Group Id. + * @param isSimpleConsumerGroup If consumer group is simple or not. + * @param state The state of the consumer group. + * @param type The type of the consumer group. + */ + public ConsumerGroupListing( + String groupId, + boolean isSimpleConsumerGroup, + Optional state, + Optional type + ) { this.groupId = groupId; this.isSimpleConsumerGroup = isSimpleConsumerGroup; this.state = Objects.requireNonNull(state); + this.type = Objects.requireNonNull(type); } /** @@ -74,42 +94,38 @@ public Optional state() { return state; } + /** + * The type of the consumer group. + * + * @return An Optional containing the type, if available. + */ + public Optional type() { + return type; + } + @Override public String toString() { return "(" + "groupId='" + groupId + '\'' + ", isSimpleConsumerGroup=" + isSimpleConsumerGroup + ", state=" + state + + ", type=" + type + ')'; } @Override public int hashCode() { - return Objects.hash(groupId, isSimpleConsumerGroup, state); + return Objects.hash(groupId, isSimpleConsumerGroup(), state, type); } @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - ConsumerGroupListing other = (ConsumerGroupListing) obj; - if (groupId == null) { - if (other.groupId != null) - return false; - } else if (!groupId.equals(other.groupId)) - return false; - if (isSimpleConsumerGroup != other.isSimpleConsumerGroup) - return false; - if (state == null) { - if (other.state != null) - return false; - } else if (!state.equals(other.state)) - return false; - return true; + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof ConsumerGroupListing)) return false; + ConsumerGroupListing that = (ConsumerGroupListing) o; + return isSimpleConsumerGroup() == that.isSimpleConsumerGroup() && + Objects.equals(groupId, that.groupId) && + Objects.equals(state, that.state) && + Objects.equals(type, that.type); } - } 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 c043ec9d91..630f1000ff 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 @@ -58,6 +58,7 @@ import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.ConsumerGroupState; +import org.apache.kafka.common.GroupType; import org.apache.kafka.common.ElectionType; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaFuture; @@ -3414,7 +3415,14 @@ ListGroupsRequest.Builder createRequest(int timeoutMs) { .stream() .map(ConsumerGroupState::toString) .collect(Collectors.toList()); - return new ListGroupsRequest.Builder(new ListGroupsRequestData().setStatesFilter(states)); + List groupTypes = options.types() + .stream() + .map(GroupType::toString) + .collect(Collectors.toList()); + return new ListGroupsRequest.Builder(new ListGroupsRequestData() + .setStatesFilter(states) + .setTypesFilter(groupTypes) + ); } private void maybeAddConsumerGroup(ListGroupsResponseData.ListedGroup group) { @@ -3424,7 +3432,15 @@ private void maybeAddConsumerGroup(ListGroupsResponseData.ListedGroup group) { final Optional state = group.groupState().equals("") ? Optional.empty() : Optional.of(ConsumerGroupState.parse(group.groupState())); - final ConsumerGroupListing groupListing = new ConsumerGroupListing(groupId, protocolType.isEmpty(), state); + final Optional type = group.groupType().equals("") + ? Optional.empty() + : Optional.of(GroupType.parse(group.groupType())); + final ConsumerGroupListing groupListing = new ConsumerGroupListing( + groupId, + protocolType.isEmpty(), + state, + type + ); results.addListing(groupListing); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupsOptions.java index 9f1f38dd4a..c240da159f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupsOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupsOptions.java @@ -22,6 +22,7 @@ import java.util.Set; import org.apache.kafka.common.ConsumerGroupState; +import org.apache.kafka.common.GroupType; import org.apache.kafka.common.annotation.InterfaceStability; /** @@ -34,20 +35,38 @@ public class ListConsumerGroupsOptions extends AbstractOptions states = Collections.emptySet(); + private Set types = Collections.emptySet(); + /** - * If states is set, only groups in these states will be returned by listConsumerGroups() + * If states is set, only groups in these states will be returned by listConsumerGroups(). * Otherwise, all groups are returned. * This operation is supported by brokers with version 2.6.0 or later. */ public ListConsumerGroupsOptions inStates(Set states) { - this.states = (states == null) ? Collections.emptySet() : new HashSet<>(states); + this.states = (states == null || states.isEmpty()) ? Collections.emptySet() : new HashSet<>(states); return this; } /** - * Returns the list of States that are requested or empty if no states have been specified + * If types is set, only groups of these types will be returned by listConsumerGroups(). + * Otherwise, all groups are returned. + */ + public ListConsumerGroupsOptions withTypes(Set types) { + this.types = (types == null || types.isEmpty()) ? Collections.emptySet() : new HashSet<>(types); + return this; + } + + /** + * Returns the list of States that are requested or empty if no states have been specified. */ public Set states() { return states; } + + /** + * Returns the list of group types that are requested or empty if no types have been specified. + */ + public Set types() { + return types; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandler.java index 2daad22603..9f40d19f00 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandler.java @@ -79,15 +79,15 @@ public static SimpleAdminApiFuture newFuture( @Override public DeleteRecordsRequest.Builder buildBatchedRequest(int brokerId, Set keys) { Map deletionsForTopic = new HashMap<>(); - for (Map.Entry entry: recordsToDelete.entrySet()) { - TopicPartition topicPartition = entry.getKey(); + for (TopicPartition topicPartition : keys) { + RecordsToDelete toDelete = recordsToDelete.get(topicPartition); DeleteRecordsRequestData.DeleteRecordsTopic deleteRecords = deletionsForTopic.computeIfAbsent( topicPartition.topic(), key -> new DeleteRecordsRequestData.DeleteRecordsTopic().setName(topicPartition.topic()) ); deleteRecords.partitions().add(new DeleteRecordsRequestData.DeleteRecordsPartition() .setPartitionIndex(topicPartition.partition()) - .setOffset(entry.getValue().beforeOffset())); + .setOffset(toDelete.beforeOffset())); } DeleteRecordsRequestData data = new DeleteRecordsRequestData() diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index d4b461b014..fcd57469c2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -36,32 +36,32 @@ import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.consumer.internals.events.AllTopicsMetadataEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; -import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.AsyncCommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; +import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.CommitApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.clients.consumer.internals.events.EventProcessor; -import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; -import org.apache.kafka.clients.consumer.internals.events.LeaveOnCloseApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsEvent; +import org.apache.kafka.clients.consumer.internals.events.LeaveOnCloseEvent; +import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; -import org.apache.kafka.clients.consumer.internals.events.PollApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.SyncCommitApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.TopicMetadataApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.PollEvent; +import org.apache.kafka.clients.consumer.internals.events.ResetPositionsEvent; +import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeEvent; +import org.apache.kafka.clients.consumer.internals.events.SyncCommitEvent; +import org.apache.kafka.clients.consumer.internals.events.TopicMetadataEvent; +import org.apache.kafka.clients.consumer.internals.events.UnsubscribeEvent; +import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsEvent; import org.apache.kafka.clients.consumer.internals.metrics.KafkaConsumerMetrics; import org.apache.kafka.clients.consumer.internals.metrics.RebalanceCallbackMetricsManager; import org.apache.kafka.common.Cluster; @@ -111,6 +111,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -179,7 +180,7 @@ public BackgroundEventProcessor(final LogContext logContext, /** * Process the events—if any—that were produced by the {@link ConsumerNetworkThread network thread}. - * It is possible that {@link org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent an error} + * It is possible that {@link ErrorEvent an error} * could occur when processing the events. In such cases, the processor will take a reference to the first * error, continue to process the remaining events, and then throw the first error that occurred. */ @@ -209,11 +210,7 @@ public boolean process() { public void process(final BackgroundEvent event) { switch (event.type()) { case ERROR: - process((ErrorBackgroundEvent) event); - break; - - case GROUP_METADATA_UPDATE: - process((GroupMetadataUpdateEvent) event); + process((ErrorEvent) event); break; case CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED: @@ -226,22 +223,10 @@ public void process(final BackgroundEvent event) { } } - private void process(final ErrorBackgroundEvent event) { + private void process(final ErrorEvent event) { throw event.error(); } - private void process(final GroupMetadataUpdateEvent event) { - if (AsyncKafkaConsumer.this.groupMetadata.isPresent()) { - final ConsumerGroupMetadata currentGroupMetadata = AsyncKafkaConsumer.this.groupMetadata.get(); - AsyncKafkaConsumer.this.groupMetadata = Optional.of(new ConsumerGroupMetadata( - currentGroupMetadata.groupId(), - event.memberEpoch(), - event.memberId(), - currentGroupMetadata.groupInstanceId() - )); - } - } - private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { ApplicationEvent invokedEvent = invokeRebalanceCallbacks( rebalanceListenerInvoker, @@ -255,7 +240,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { private final ApplicationEventHandler applicationEventHandler; private final Time time; - private Optional groupMetadata = Optional.empty(); + private final AtomicReference> groupMetadata = new AtomicReference<>(Optional.empty()); private final KafkaConsumerMetrics kafkaConsumerMetrics; private Logger log; private final String clientId; @@ -287,6 +272,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { private boolean cachedSubscriptionHasAllFetchPositions; private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; + private final AtomicBoolean asyncCommitFenced; // currentThread holds the threadId of the current thread accessing the AsyncKafkaConsumer // and is used to prevent multithreaded access @@ -369,6 +355,8 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { fetchMetricsManager, clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null)); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); + this.asyncCommitFenced = new AtomicBoolean(false); + this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig)); final Supplier requestManagersSupplier = RequestManagers.supplier(time, logContext, backgroundEventHandler, @@ -382,7 +370,9 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { networkClientDelegateSupplier, clientTelemetryReporter, metrics, - offsetCommitCallbackInvoker); + offsetCommitCallbackInvoker, + this::updateGroupMetadata + ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, applicationEventQueue, @@ -412,8 +402,6 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) ); - this.groupMetadata = initializeGroupMetadata(config, groupRebalanceConfig); - // The FetchCollector is only used on the application thread. this.fetchCollector = fetchCollectorFactory.build(logContext, metadata, @@ -425,7 +413,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX); - if (groupMetadata.isPresent() && + if (groupMetadata.get().isPresent() && GroupProtocol.of(config.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)) == GroupProtocol.CONSUMER) { config.ignore(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG); // Used by background thread } @@ -477,7 +465,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { rebalanceListenerInvoker ); this.metrics = metrics; - this.groupMetadata = initializeGroupMetadata(groupId, Optional.empty()); + this.groupMetadata.set(initializeGroupMetadata(groupId, Optional.empty())); this.metadata = metadata; this.retryBackoffMs = retryBackoffMs; this.defaultApiTimeoutMs = defaultApiTimeoutMs; @@ -488,6 +476,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.clientTelemetryReporter = Optional.empty(); this.autoCommitEnabled = autoCommitEnabled; this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); + this.asyncCommitFenced = new AtomicBoolean(false); } AsyncKafkaConsumer(LogContext logContext, @@ -531,7 +520,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { GroupRebalanceConfig.ProtocolType.CONSUMER ); - this.groupMetadata = initializeGroupMetadata(config, groupRebalanceConfig); + this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig)); BlockingQueue applicationEventQueue = new LinkedBlockingQueue<>(); BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); @@ -553,6 +542,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { client ); this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors); + this.asyncCommitFenced = new AtomicBoolean(false); Supplier requestManagersSupplier = RequestManagers.supplier( time, logContext, @@ -567,7 +557,8 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { networkClientDelegateSupplier, clientTelemetryReporter, metrics, - offsetCommitCallbackInvoker + offsetCommitCallbackInvoker, + this::updateGroupMetadata ); Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, @@ -650,17 +641,35 @@ private Optional initializeGroupMetadata(final String gro throw new InvalidGroupIdException("The configured " + ConsumerConfig.GROUP_ID_CONFIG + " should not be an empty string or whitespace."); } else { - return Optional.of(new ConsumerGroupMetadata( - groupId, - JoinGroupRequest.UNKNOWN_GENERATION_ID, - JoinGroupRequest.UNKNOWN_MEMBER_ID, - groupInstanceId - )); + return Optional.of(initializeConsumerGroupMetadata(groupId, groupInstanceId)); } } return Optional.empty(); } + private ConsumerGroupMetadata initializeConsumerGroupMetadata(final String groupId, + final Optional groupInstanceId) { + return new ConsumerGroupMetadata( + groupId, + JoinGroupRequest.UNKNOWN_GENERATION_ID, + JoinGroupRequest.UNKNOWN_MEMBER_ID, + groupInstanceId + ); + } + + private void updateGroupMetadata(final Optional memberEpoch, final Optional memberId) { + groupMetadata.updateAndGet( + oldGroupMetadataOptional -> oldGroupMetadataOptional.map( + oldGroupMetadata -> new ConsumerGroupMetadata( + oldGroupMetadata.groupId(), + memberEpoch.orElse(oldGroupMetadata.generationId()), + memberId.orElse(oldGroupMetadata.memberId()), + oldGroupMetadata.groupInstanceId() + ) + ) + ); + } + /** * poll implementation using {@link ApplicationEventHandler}. * 1. Poll for background events. If there's a fetch response event, process the record and return it. If it is @@ -703,7 +712,7 @@ public ConsumerRecords poll(final Duration timeout) { do { // Make sure to let the background thread know that we are still polling. - applicationEventHandler.add(new PollApplicationEvent(timer.currentTimeMs())); + applicationEventHandler.add(new PollEvent(timer.currentTimeMs())); // We must not allow wake-ups between polling for fetches and returning the records. // If the polled fetches are not empty the consumed position has already been updated in the polling @@ -712,18 +721,14 @@ public ConsumerRecords poll(final Duration timeout) { wakeupTrigger.maybeTriggerWakeup(); updateAssignmentMetadataIfNeeded(timer); - if (isGenerationKnownOrPartitionsUserAssigned()) { - final Fetch fetch = pollForFetches(timer); - if (!fetch.isEmpty()) { - if (fetch.records().isEmpty()) { - log.trace("Returning empty records from `poll()` " - + "since the consumer's position has advanced for at least one topic partition"); - } - - return interceptors.onConsume(new ConsumerRecords<>(fetch.records())); + final Fetch fetch = pollForFetches(timer); + if (!fetch.isEmpty()) { + if (fetch.records().isEmpty()) { + log.trace("Returning empty records from `poll()` " + + "since the consumer's position has advanced for at least one topic partition"); } - } else { - timer.update(); + + return interceptors.onConsume(new ConsumerRecords<>(fetch.records())); } // We will wait for retryBackoffMs } while (timer.notExpired()); @@ -735,13 +740,6 @@ public ConsumerRecords poll(final Duration timeout) { } } - private boolean isGenerationKnownOrPartitionsUserAssigned() { - if (subscriptions.hasAutoAssignedPartitions()) { - return groupMetadata.filter(g -> g.generationId() != JoinGroupRequest.UNKNOWN_GENERATION_ID).isPresent(); - } - return true; - } - /** * Commit offsets returned on the last {@link #poll(Duration) poll()} for all the subscribed list of topics and * partitions. @@ -768,7 +766,7 @@ public void commitAsync(OffsetCommitCallback callback) { public void commitAsync(Map offsets, OffsetCommitCallback callback) { acquireAndEnsureOpen(); try { - AsyncCommitApplicationEvent asyncCommitEvent = new AsyncCommitApplicationEvent(offsets); + AsyncCommitEvent asyncCommitEvent = new AsyncCommitEvent(offsets); CompletableFuture future = commit(asyncCommitEvent); future.whenComplete((r, t) -> { @@ -776,6 +774,10 @@ public void commitAsync(Map offsets, OffsetCo offsetCommitCallbackInvoker.enqueueInterceptorInvocation(offsets); } + if (t instanceof FencedInstanceIdException) { + asyncCommitFenced.set(true); + } + if (callback == null) { if (t != null) { log.error("Offset commit with offsets {} failed", offsets, t); @@ -790,9 +792,9 @@ public void commitAsync(Map offsets, OffsetCo } } - private CompletableFuture commit(final CommitApplicationEvent commitEvent) { - maybeInvokeCommitCallbacks(); + private CompletableFuture commit(final CommitEvent commitEvent) { maybeThrowFencedInstanceException(); + maybeInvokeCommitCallbacks(); maybeThrowInvalidGroupIdException(); Map offsets = commitEvent.offsets(); @@ -936,13 +938,14 @@ public Map committed(final Set committedOffsets = applicationEventHandler.addAndGet(event, - time.timer(timeout)); + timer); committedOffsets.forEach(this::updateLastSeenEpochIfNewer); return committedOffsets; } catch (TimeoutException e) { @@ -959,7 +962,7 @@ public Map committed(final Set partitionsFor(String topic, Duration timeout) { throw new TimeoutException(); } - final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(topic, timeout.toMillis()); - wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); + final Timer timer = time.timer(timeout); + final TopicMetadataEvent topicMetadataEvent = new TopicMetadataEvent(topic, timer); + wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { Map> topicMetadata = - applicationEventHandler.addAndGet(topicMetadataApplicationEvent, time.timer(timeout)); + applicationEventHandler.addAndGet(topicMetadataEvent, timer); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { @@ -1017,11 +1020,11 @@ public Map> listTopics(Duration timeout) { throw new TimeoutException(); } - final TopicMetadataApplicationEvent topicMetadataApplicationEvent = - new TopicMetadataApplicationEvent(timeout.toMillis()); - wakeupTrigger.setActiveTask(topicMetadataApplicationEvent.future()); + final Timer timer = time.timer(timeout); + final AllTopicsMetadataEvent topicMetadataEvent = new AllTopicsMetadataEvent(timer); + wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { - return applicationEventHandler.addAndGet(topicMetadataApplicationEvent, time.timer(timeout)); + return applicationEventHandler.addAndGet(topicMetadataEvent, timer); } finally { wakeupTrigger.clearTask(); } @@ -1089,16 +1092,18 @@ public Map offsetsForTimes(Map beginningOrEndOffset(Collection timestampToSearch = partitions .stream() .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); - ListOffsetsApplicationEvent listOffsetsEvent = new ListOffsetsApplicationEvent( + Timer timer = time.timer(timeout); + ListOffsetsEvent listOffsetsEvent = new ListOffsetsEvent( timestampToSearch, - false); + false, + timer); Map offsetAndTimestampMap = applicationEventHandler.addAndGet( listOffsetsEvent, - time.timer(timeout)); + timer); return offsetAndTimestampMap .entrySet() .stream() @@ -1187,7 +1194,7 @@ public ConsumerGroupMetadata groupMetadata() { acquireAndEnsureOpen(); try { maybeThrowInvalidGroupIdException(); - return groupMetadata.get(); + return groupMetadata.get().get(); } finally { release(); } @@ -1229,6 +1236,9 @@ private void close(Duration timeout, boolean swallowException) { log.trace("Closing the Kafka consumer"); AtomicReference firstException = new AtomicReference<>(); + // We are already closing with a timeout, don't allow wake-ups from here on. + wakeupTrigger.disableWakeups(); + final Timer closeTimer = time.timer(timeout); clientTelemetryReporter.ifPresent(reporter -> reporter.initiateClose(timeout.toMillis())); closeTimer.update(); @@ -1264,22 +1274,21 @@ private void close(Duration timeout, boolean swallowException) { * 3. if partition revocation completes successfully, send leave group */ void prepareShutdown(final Timer timer, final AtomicReference firstException) { - if (!groupMetadata.isPresent()) + if (!groupMetadata.get().isPresent()) return; - maybeAutoCommitSync(autoCommitEnabled, timer, firstException); - applicationEventHandler.add(new CommitOnCloseApplicationEvent()); + maybeAutoCommitSync(autoCommitEnabled, timer); + applicationEventHandler.add(new CommitOnCloseEvent()); completeQuietly( () -> { maybeRevokePartitions(); - applicationEventHandler.addAndGet(new LeaveOnCloseApplicationEvent(), timer); + applicationEventHandler.addAndGet(new LeaveOnCloseEvent(timer), timer); }, "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); } // Visible for testing void maybeAutoCommitSync(final boolean shouldAutoCommit, - final Timer timer, - final AtomicReference firstException) { + final Timer timer) { if (!shouldAutoCommit) return; Map allConsumed = subscriptions.allConsumed(); @@ -1349,7 +1358,7 @@ public void commitSync(Map offsets, Duration long commitStart = time.nanoseconds(); try { Timer requestTimer = time.timer(timeout.toMillis()); - SyncCommitApplicationEvent syncCommitEvent = new SyncCommitApplicationEvent(offsets, timeout.toMillis()); + SyncCommitEvent syncCommitEvent = new SyncCommitEvent(offsets, requestTimer); CompletableFuture commitFuture = commit(syncCommitEvent); wakeupTrigger.setActiveTask(commitFuture); ConsumerUtils.getResult(commitFuture, requestTimer); @@ -1429,7 +1438,7 @@ public void assign(Collection partitions) { // be no following rebalance. // // See the ApplicationEventProcessor.process() method that handles this event for more detail. - applicationEventHandler.add(new AssignmentChangeApplicationEvent(subscriptions.allConsumed(), time.milliseconds())); + applicationEventHandler.add(new AssignmentChangeEvent(subscriptions.allConsumed(), time.milliseconds())); log.info("Assigned to partition(s): {}", join(partitions, ", ")); if (subscriptions.assignFromUser(new HashSet<>(partitions))) @@ -1462,19 +1471,19 @@ public void unsubscribe() { acquireAndEnsureOpen(); try { fetchBuffer.retainAll(Collections.emptySet()); - if (groupMetadata.isPresent()) { - UnsubscribeApplicationEvent unsubscribeApplicationEvent = new UnsubscribeApplicationEvent(); - applicationEventHandler.add(unsubscribeApplicationEvent); - log.info("Unsubscribing all topics or patterns and assigned partitions"); + if (groupMetadata.get().isPresent()) { Timer timer = time.timer(Long.MAX_VALUE); + UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(timer); + applicationEventHandler.add(unsubscribeEvent); + log.info("Unsubscribing all topics or patterns and assigned partitions"); try { - processBackgroundEvents(backgroundEventProcessor, unsubscribeApplicationEvent.future(), timer); + processBackgroundEvents(backgroundEventProcessor, unsubscribeEvent.future(), timer); log.info("Unsubscribed all topics or patterns and assigned partitions"); } catch (TimeoutException e) { log.error("Failed while waiting for the unsubscribe event to complete"); } - groupMetadata = initializeGroupMetadata(groupMetadata.get().groupId(), Optional.empty()); + resetGroupMetadata(); } subscriptions.unsubscribe(); } finally { @@ -1482,6 +1491,16 @@ public void unsubscribe() { } } + private void resetGroupMetadata() { + groupMetadata.updateAndGet( + oldGroupMetadataOptional -> oldGroupMetadataOptional + .map(oldGroupMetadata -> initializeConsumerGroupMetadata( + oldGroupMetadata.groupId(), + oldGroupMetadata.groupInstanceId() + )) + ); + } + @Override @Deprecated public ConsumerRecords poll(final long timeoutMs) { @@ -1567,7 +1586,7 @@ private boolean updateFetchPositions(final Timer timer) { // Validate positions using the partition leader end offsets, to detect if any partition // has been truncated due to a leader change. This will trigger an OffsetForLeaderEpoch // request, retrieve the partition end offsets, and validate the current position against it. - applicationEventHandler.addAndGet(new ValidatePositionsApplicationEvent(), timer); + applicationEventHandler.addAndGet(new ValidatePositionsEvent(timer), timer); cachedSubscriptionHasAllFetchPositions = subscriptions.hasAllFetchPositions(); if (cachedSubscriptionHasAllFetchPositions) return true; @@ -1590,7 +1609,7 @@ private boolean updateFetchPositions(final Timer timer) { // which are awaiting reset. This will trigger a ListOffset request, retrieve the // partition offsets according to the strategy (ex. earliest, latest), and update the // positions. - applicationEventHandler.addAndGet(new ResetPositionsApplicationEvent(), timer); + applicationEventHandler.addAndGet(new ResetPositionsEvent(timer), timer); return true; } catch (TimeoutException e) { return false; @@ -1603,7 +1622,7 @@ private boolean updateFetchPositions(final Timer timer) { * according to config {@link CommonClientConfigs#GROUP_ID_CONFIG} */ private boolean isCommittedOffsetsManagementEnabled() { - return groupMetadata.isPresent(); + return groupMetadata.get().isPresent(); } /** @@ -1620,10 +1639,10 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { log.debug("Refreshing committed offsets for partitions {}", initializingPartitions); try { - final FetchCommittedOffsetsApplicationEvent event = - new FetchCommittedOffsetsApplicationEvent( + final FetchCommittedOffsetsEvent event = + new FetchCommittedOffsetsEvent( initializingPartitions, - timer.remainingMs()); + timer); final Map offsets = applicationEventHandler.addAndGet(event, timer); refreshCommittedOffsets(offsets, metadata, subscriptions); return true; @@ -1646,8 +1665,8 @@ private void updateLastSeenEpochIfNewer(TopicPartition topicPartition, OffsetAnd @Override public boolean updateAssignmentMetadataIfNeeded(Timer timer) { - maybeInvokeCommitCallbacks(); maybeThrowFencedInstanceException(); + maybeInvokeCommitCallbacks(); backgroundEventProcessor.process(); // Keeping this updateAssignmentMetadataIfNeeded wrapping up the updateFetchPositions as @@ -1770,7 +1789,7 @@ private void subscribeInternal(Collection topics, Optional topics, Optional * * As an example, take {@link #unsubscribe()}. To start unsubscribing, the application thread enqueues an - * {@link UnsubscribeApplicationEvent} on the application event queue. That event will eventually trigger the + * {@link UnsubscribeEvent} on the application event queue. That event will eventually trigger the * rebalancing logic in the background thread. Critically, as part of this rebalancing work, the * {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} callback needs to be invoked. However, * this callback must be executed on the application thread. To achieve this, the background thread enqueues a @@ -1902,14 +1921,14 @@ public KafkaConsumerMetrics kafkaConsumerMetrics() { } private void maybeThrowFencedInstanceException() { - if (offsetCommitCallbackInvoker.hasFencedException()) { + if (asyncCommitFenced.get()) { String groupInstanceId = "unknown"; - if (!groupMetadata.isPresent()) { + if (!groupMetadata.get().isPresent()) { log.error("No group metadata found although a group ID was provided. This is a bug!"); - } else if (!groupMetadata.get().groupInstanceId().isPresent()) { + } else if (!groupMetadata.get().get().groupInstanceId().isPresent()) { log.error("No group instance ID found although the consumer is fenced. This is a bug!"); } else { - groupInstanceId = groupMetadata.get().groupInstanceId().get(); + groupInstanceId = groupMetadata.get().get().groupInstanceId().get(); } throw new FencedInstanceIdException("Get fenced exception for group.instance.id " + groupInstanceId); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index 9206783d56..67ea6eb313 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -30,6 +30,7 @@ import org.apache.kafka.common.errors.StaleMemberEpochException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.errors.UnstableOffsetCommitException; import org.apache.kafka.common.message.OffsetCommitRequestData; import org.apache.kafka.common.message.OffsetCommitResponseData; @@ -272,14 +273,18 @@ private void maybeResetTimerWithBackoff(final CompletableFuture maybeAutoCommitSyncNow(final long retryExpirationTimeMs) { + public CompletableFuture maybeAutoCommitSyncBeforeRevocation(final long retryExpirationTimeMs) { if (!autoCommitEnabled()) { return CompletableFuture.completedFuture(null); } @@ -287,12 +292,12 @@ public CompletableFuture maybeAutoCommitSyncNow(final long retryExpiration CompletableFuture result = new CompletableFuture<>(); OffsetCommitRequestState requestState = createOffsetCommitRequest(subscriptions.allConsumed(), Optional.of(retryExpirationTimeMs)); - autoCommitSyncNowWithRetries(requestState, result); + autoCommitSyncBeforeRevocationWithRetries(requestState, result); return result; } - private void autoCommitSyncNowWithRetries(OffsetCommitRequestState requestAttempt, - CompletableFuture result) { + private void autoCommitSyncBeforeRevocationWithRetries(OffsetCommitRequestState requestAttempt, + CompletableFuture result) { CompletableFuture> commitAttempt = requestAutoCommit(requestAttempt); commitAttempt.whenComplete((committedOffsets, error) -> { if (error == null) { @@ -300,16 +305,19 @@ private void autoCommitSyncNowWithRetries(OffsetCommitRequestState requestAttemp } else { if (error instanceof RetriableException || isStaleEpochErrorAndValidEpochAvailable(error)) { if (error instanceof TimeoutException && requestAttempt.isExpired) { - log.debug("Auto-commit sync timed out and won't be retried anymore"); + log.debug("Auto-commit sync before revocation timed out and won't be retried anymore"); + result.completeExceptionally(error); + } else if (error instanceof UnknownTopicOrPartitionException) { + log.debug("Auto-commit sync before revocation failed because topic or partition were deleted"); result.completeExceptionally(error); } else { // Make sure the auto-commit is retries with the latest offsets requestAttempt.offsets = subscriptions.allConsumed(); requestAttempt.resetFuture(); - autoCommitSyncNowWithRetries(requestAttempt, result); + autoCommitSyncBeforeRevocationWithRetries(requestAttempt, result); } } else { - log.debug("Auto-commit sync failed with non-retriable error", error); + log.debug("Auto-commit sync before revocation failed with non-retriable error", error); result.completeExceptionally(error); } } @@ -438,7 +446,7 @@ private Throwable commitSyncExceptionForError(Throwable error) { private Throwable commitAsyncExceptionForError(Throwable error) { if (error instanceof RetriableException) { - return new RetriableCommitFailedException(error.getMessage()); + return new RetriableCommitFailedException(error); } return error; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java index d6a72812a5..a6cc28fb0f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java @@ -17,7 +17,7 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.GroupAuthorizationException; @@ -175,12 +175,12 @@ private void onFailedResponse(final long currentTimeMs, final Throwable exceptio if (exception == Errors.GROUP_AUTHORIZATION_FAILED.exception()) { log.debug("FindCoordinator request failed due to authorization error {}", exception.getMessage()); KafkaException groupAuthorizationException = GroupAuthorizationException.forGroupId(this.groupId); - backgroundEventHandler.add(new ErrorBackgroundEvent(groupAuthorizationException)); + backgroundEventHandler.add(new ErrorEvent(groupAuthorizationException)); return; } log.warn("FindCoordinator request failed due to fatal exception", exception); - backgroundEventHandler.add(new ErrorBackgroundEvent(exception)); + backgroundEventHandler.add(new ErrorEvent(exception)); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java index 692a67b9c3..998136797b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java @@ -248,10 +248,10 @@ private CompletedFetch handleInitializeSuccess(final CompletedFetch completedFet // we are interested in this fetch only if the beginning offset matches the // current consumed position - SubscriptionState.FetchPosition position = subscriptions.position(tp); + SubscriptionState.FetchPosition position = subscriptions.positionOrNull(tp); if (position == null || position.offset != fetchOffset) { log.debug("Discarding stale fetch response for partition {} since its offset {} does not match " + - "the expected offset {}", tp, fetchOffset, position); + "the expected offset {} or the partition has been unassigned", tp, fetchOffset, position); return null; } @@ -278,32 +278,48 @@ private CompletedFetch handleInitializeSuccess(final CompletedFetch completedFet } } - if (partition.highWatermark() >= 0) { - log.trace("Updating high watermark for partition {} to {}", tp, partition.highWatermark()); - subscriptions.updateHighWatermark(tp, partition.highWatermark()); + if (!updatePartitionState(partition, tp)) { + return null; + } + + completedFetch.setInitialized(); + return completedFetch; + } + + private boolean updatePartitionState(final FetchResponseData.PartitionData partitionData, + final TopicPartition tp) { + if (partitionData.highWatermark() >= 0) { + log.trace("Updating high watermark for partition {} to {}", tp, partitionData.highWatermark()); + if (!subscriptions.tryUpdatingHighWatermark(tp, partitionData.highWatermark())) { + return false; + } } - if (partition.logStartOffset() >= 0) { - log.trace("Updating log start offset for partition {} to {}", tp, partition.logStartOffset()); - subscriptions.updateLogStartOffset(tp, partition.logStartOffset()); + if (partitionData.logStartOffset() >= 0) { + log.trace("Updating log start offset for partition {} to {}", tp, partitionData.logStartOffset()); + if (!subscriptions.tryUpdatingLogStartOffset(tp, partitionData.logStartOffset())) { + return false; + } } - if (partition.lastStableOffset() >= 0) { - log.trace("Updating last stable offset for partition {} to {}", tp, partition.lastStableOffset()); - subscriptions.updateLastStableOffset(tp, partition.lastStableOffset()); + if (partitionData.lastStableOffset() >= 0) { + log.trace("Updating last stable offset for partition {} to {}", tp, partitionData.lastStableOffset()); + if (!subscriptions.tryUpdatingLastStableOffset(tp, partitionData.lastStableOffset())) { + return false; + } } - if (FetchResponse.isPreferredReplica(partition)) { - subscriptions.updatePreferredReadReplica(completedFetch.partition, partition.preferredReadReplica(), () -> { - long expireTimeMs = time.milliseconds() + metadata.metadataExpireMs(); - log.debug("Updating preferred read replica for partition {} to {}, set to expire at {}", - tp, partition.preferredReadReplica(), expireTimeMs); - return expireTimeMs; - }); + if (FetchResponse.isPreferredReplica(partitionData)) { + return subscriptions.tryUpdatingPreferredReadReplica( + tp, partitionData.preferredReadReplica(), () -> { + long expireTimeMs = time.milliseconds() + metadata.metadataExpireMs(); + log.debug("Updating preferred read replica for partition {} to {}, set to expire at {}", + tp, partitionData.preferredReadReplica(), expireTimeMs); + return expireTimeMs; + }); } - completedFetch.setInitialized(); - return completedFetch; + return true; } private void handleInitializeErrors(final CompletedFetch completedFetch, final Errors error) { @@ -331,17 +347,17 @@ private void handleInitializeErrors(final CompletedFetch completedFetch, final E if (!clearedReplicaId.isPresent()) { // If there's no preferred replica to clear, we're fetching from the leader so handle this error normally - SubscriptionState.FetchPosition position = subscriptions.position(tp); + SubscriptionState.FetchPosition position = subscriptions.positionOrNull(tp); if (position == null || fetchOffset != position.offset) { log.debug("Discarding stale fetch response for partition {} since the fetched offset {} " + - "does not match the current offset {}", tp, fetchOffset, position); + "does not match the current offset {} or the partition has been unassigned", tp, fetchOffset, position); } else { String errorMessage = "Fetch position " + position + " is out of range for partition " + tp; if (subscriptions.hasDefaultOffsetResetPolicy()) { log.info("{}, resetting offset", errorMessage); - subscriptions.requestOffsetReset(tp); + subscriptions.requestOffsetResetIfPartitionAssigned(tp); } else { log.info("{}, raising error to the application since no reset policy is configured", errorMessage); throw new OffsetOutOfRangeException(errorMessage, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java index 550e5b92eb..5b54e8a436 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java @@ -18,11 +18,11 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.internals.MembershipManager.LocalAssignment; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.clients.consumer.internals.metrics.HeartbeatMetricsManager; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.GroupAuthorizationException; @@ -110,7 +110,6 @@ public class HeartbeatRequestManager implements RequestManager { * sending heartbeat until the next poll. */ private final Timer pollTimer; - private GroupMetadataUpdateEvent previousGroupMetadataUpdateEvent = null; /** * Holding the heartbeat sensor to measure heartbeat timing and response latency @@ -327,28 +326,12 @@ private void onResponse(final ConsumerGroupHeartbeatResponse response, long curr heartbeatRequestState.updateHeartbeatIntervalMs(response.data().heartbeatIntervalMs()); heartbeatRequestState.onSuccessfulAttempt(currentTimeMs); heartbeatRequestState.resetTimer(); - membershipManager.onHeartbeatResponseReceived(response.data()); - maybeSendGroupMetadataUpdateEvent(); + membershipManager.onHeartbeatSuccess(response.data()); return; } onErrorResponse(response, currentTimeMs); } - private void maybeSendGroupMetadataUpdateEvent() { - if (previousGroupMetadataUpdateEvent == null || - !previousGroupMetadataUpdateEvent.memberId().equals(membershipManager.memberId()) || - previousGroupMetadataUpdateEvent.memberEpoch() != membershipManager.memberEpoch()) { - - final GroupMetadataUpdateEvent currentGroupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - membershipManager.memberEpoch(), - previousGroupMetadataUpdateEvent != null && membershipManager.memberId() == null ? - previousGroupMetadataUpdateEvent.memberId() : membershipManager.memberId() - ); - this.backgroundEventHandler.add(currentGroupMetadataUpdateEvent); - previousGroupMetadataUpdateEvent = currentGroupMetadataUpdateEvent; - } - } - private void onErrorResponse(final ConsumerGroupHeartbeatResponse response, final long currentTimeMs) { Errors error = Errors.forCode(response.data().errorCode()); @@ -357,6 +340,7 @@ private void onErrorResponse(final ConsumerGroupHeartbeatResponse response, this.heartbeatState.reset(); this.heartbeatRequestState.onFailedAttempt(currentTimeMs); + membershipManager.onHeartbeatFailure(); switch (error) { case NOT_COORDINATOR: @@ -445,7 +429,7 @@ private void logInfo(final String message, } private void handleFatalFailure(Throwable error) { - backgroundEventHandler.add(new ErrorBackgroundEvent(error)); + backgroundEventHandler.add(new ErrorEvent(error)); membershipManager.transitionToFatal(); } @@ -546,36 +530,32 @@ public ConsumerGroupHeartbeatRequestData buildRequestData() { // MemberEpoch - always sent data.setMemberEpoch(membershipManager.memberEpoch()); - // InstanceId - only sent if has changed since the last heartbeat - // Always send when leaving the group as a static member - membershipManager.groupInstanceId().ifPresent(groupInstanceId -> { - if (!groupInstanceId.equals(sentFields.instanceId) || membershipManager.memberEpoch() == ConsumerGroupHeartbeatRequest.LEAVE_GROUP_STATIC_MEMBER_EPOCH) { - data.setInstanceId(groupInstanceId); - sentFields.instanceId = groupInstanceId; - } - }); + // InstanceId - set if present + membershipManager.groupInstanceId().ifPresent(data::setInstanceId); + + boolean sendAllFields = membershipManager.state() == MemberState.JOINING; - // RebalanceTimeoutMs - only sent if has changed since the last heartbeat - if (sentFields.rebalanceTimeoutMs != rebalanceTimeoutMs) { + // RebalanceTimeoutMs - only sent when joining or if it has changed since the last heartbeat + if (sendAllFields || sentFields.rebalanceTimeoutMs != rebalanceTimeoutMs) { data.setRebalanceTimeoutMs(rebalanceTimeoutMs); sentFields.rebalanceTimeoutMs = rebalanceTimeoutMs; } if (!this.subscriptions.hasPatternSubscription()) { - // SubscribedTopicNames - only sent if has changed since the last heartbeat + // SubscribedTopicNames - only sent when joining or if it has changed since the last heartbeat TreeSet subscribedTopicNames = new TreeSet<>(this.subscriptions.subscription()); - if (!subscribedTopicNames.equals(sentFields.subscribedTopicNames)) { + if (sendAllFields || !subscribedTopicNames.equals(sentFields.subscribedTopicNames)) { data.setSubscribedTopicNames(new ArrayList<>(this.subscriptions.subscription())); sentFields.subscribedTopicNames = subscribedTopicNames; } } else { - // SubscribedTopicRegex - only sent if has changed since the last heartbeat + // SubscribedTopicRegex - only sent if it has changed since the last heartbeat // - not supported yet } - // ServerAssignor - only sent if has changed since the last heartbeat + // ServerAssignor - sent when joining or if it has changed since the last heartbeat this.membershipManager.serverAssignor().ifPresent(serverAssignor -> { - if (!serverAssignor.equals(sentFields.serverAssignor)) { + if (sendAllFields || !serverAssignor.equals(sentFields.serverAssignor)) { data.setServerAssignor(serverAssignor); sentFields.serverAssignor = serverAssignor; } @@ -583,18 +563,16 @@ public ConsumerGroupHeartbeatRequestData buildRequestData() { // ClientAssignors - not supported yet - // TopicPartitions - only sent if it has changed since the last heartbeat. Note that - // the string consists of just the topic ID and the partitions. When an assignment is - // received, we might not yet know the topic name, and then it is learnt subsequently - // by a metadata update. - TreeSet assignedPartitions = membershipManager.currentAssignment().entrySet().stream() - .map(entry -> entry.getKey() + "-" + entry.getValue()) - .collect(Collectors.toCollection(TreeSet::new)); - if (!assignedPartitions.equals(sentFields.topicPartitions)) { + // TopicPartitions - sent when joining or with the first heartbeat after a new assignment from + // the server was reconciled. This is ensured by resending the topic partitions whenever the + // local assignment, including its local epoch is changed (although the local epoch is not sent + // in the heartbeat). + LocalAssignment local = membershipManager.currentAssignment(); + if (sendAllFields || !local.equals(sentFields.localAssignment)) { List topicPartitions = - buildTopicPartitionsList(membershipManager.currentAssignment()); + buildTopicPartitionsList(local.partitions); data.setTopicPartitions(topicPartitions); - sentFields.topicPartitions = assignedPartitions; + sentFields.localAssignment = local; } return data; @@ -610,20 +588,18 @@ private List buildTopicPartit // Fields of ConsumerHeartbeatRequest sent in the most recent request static class SentFields { - private String instanceId = null; private int rebalanceTimeoutMs = -1; private TreeSet subscribedTopicNames = null; private String serverAssignor = null; - private TreeSet topicPartitions = null; + private LocalAssignment localAssignment = null; SentFields() {} void reset() { - instanceId = null; - rebalanceTimeoutMs = -1; subscribedTopicNames = null; + rebalanceTimeoutMs = -1; serverAssignor = null; - topicPartitions = null; + localAssignment = null; } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java index a9c23d7b4d..c33ac396d2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java @@ -18,12 +18,17 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; +import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; +import java.util.Collections; +import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.SortedSet; +import java.util.TreeSet; import java.util.concurrent.CompletableFuture; /** @@ -68,7 +73,12 @@ public interface MembershipManager extends RequestManager { * * @param response Heartbeat response to extract member info and errors from. */ - void onHeartbeatResponseReceived(ConsumerGroupHeartbeatResponseData response); + void onHeartbeatSuccess(ConsumerGroupHeartbeatResponseData response); + + /** + * Notify the member that an error heartbeat response was received. + */ + void onHeartbeatFailure(); /** * Update state when a heartbeat is sent out. This will transition out of the states that end @@ -94,7 +104,7 @@ public interface MembershipManager extends RequestManager { * @return Current assignment for the member as received from the broker (topic IDs and * partitions). This is the last assignment that the member has successfully reconciled. */ - Map> currentAssignment(); + LocalAssignment currentAssignment(); /** * Transition the member to the FENCED state, where the member will release the assignment by @@ -180,4 +190,88 @@ public interface MembershipManager extends RequestManager { * releasing its assignment. This is expected to be used when the poll timer is reset. */ void maybeRejoinStaleMember(); + + /** + * A data structure to represent the current assignment, and current target assignment of a member in a consumer group. + * + * Besides the assigned partitions, it contains a local epoch that is bumped whenever the assignment changes, to ensure + * that two assignments with the same partitions but different local epochs are not considered equal. + */ + final class LocalAssignment { + + public static final long NONE_EPOCH = -1; + + public static final LocalAssignment NONE = new LocalAssignment(NONE_EPOCH, Collections.emptyMap()); + + public final long localEpoch; + + public final Map> partitions; + + public LocalAssignment(long localEpoch, Map> partitions) { + this.localEpoch = localEpoch; + this.partitions = partitions; + if (localEpoch == NONE_EPOCH && !partitions.isEmpty()) { + throw new IllegalArgumentException("Local epoch must be set if there are partitions"); + } + } + + public LocalAssignment(long localEpoch, SortedSet topicIdPartitions) { + this.localEpoch = localEpoch; + this.partitions = new HashMap<>(); + if (localEpoch == NONE_EPOCH && !topicIdPartitions.isEmpty()) { + throw new IllegalArgumentException("Local epoch must be set if there are partitions"); + } + topicIdPartitions.forEach(topicIdPartition -> { + Uuid topicId = topicIdPartition.topicId(); + partitions.computeIfAbsent(topicId, k -> new TreeSet<>()).add(topicIdPartition.partition()); + }); + } + + public String toString() { + return "LocalAssignment{" + + "localEpoch=" + localEpoch + + ", partitions=" + partitions + + '}'; + } + + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final LocalAssignment that = (LocalAssignment) o; + return localEpoch == that.localEpoch && Objects.equals(partitions, that.partitions); + } + + public int hashCode() { + return Objects.hash(localEpoch, partitions); + } + + public boolean isNone() { + return localEpoch == NONE_EPOCH; + } + + Optional updateWith(ConsumerGroupHeartbeatResponseData.Assignment assignment) { + // Return if we have an assignment, and it is the same as current assignment; comparison without creating a new collection + if (localEpoch != NONE_EPOCH) { + if (partitions.size() == assignment.topicPartitions().size() && + assignment.topicPartitions().stream().allMatch( + tp -> partitions.containsKey(tp.topicId()) && + partitions.get(tp.topicId()).size() == tp.partitions().size() && + partitions.get(tp.topicId()).containsAll(tp.partitions()))) { + return Optional.empty(); + } + } + + // Bump local epoch and replace assignment + long nextLocalEpoch = localEpoch + 1; + HashMap> partitions = new HashMap<>(); + assignment.topicPartitions().forEach(topicPartitions -> + partitions.put(topicPartitions.topicId(), new TreeSet<>(topicPartitions.partitions()))); + return Optional.of(new LocalAssignment(nextLocalEpoch, partitions)); + + } + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index a9b0f3b94d..4710746995 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -26,12 +26,14 @@ import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; +import org.apache.kafka.clients.consumer.internals.metrics.RebalanceMetricsManager; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; +import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest; import org.apache.kafka.common.telemetry.internals.ClientTelemetryProvider; @@ -55,6 +57,7 @@ import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; +import static java.util.Collections.unmodifiableList; import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED; import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_LOST; import static org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED; @@ -169,9 +172,12 @@ public class MembershipManagerImpl implements MembershipManager { private final Optional serverAssignor; /** - * Assignment that the member received from the server and successfully processed. + * Assignment that the member received from the server and successfully processed, together with + * its local epoch. + * + * This is equal to LocalAssignment.NONE when we are not in a group, or we haven't reconciled any assignment yet. */ - private Map> currentAssignment; + private LocalAssignment currentAssignment; /** * Subscription state object holding the current assignment the member has for the topics it @@ -207,12 +213,12 @@ public class MembershipManagerImpl implements MembershipManager { private final Map assignedTopicNamesCache; /** - * Topic IDs and partitions received in the last target assignment. Items are added to this set - * every time a target assignment is received. This is where the member collects the assignment - * received from the broker, even though it may not be ready to fully reconcile due to missing - * metadata. + * Topic IDs and partitions received in the last target assignment, together with its local epoch. + * + * This member variable is reassigned every time a new assignment is received. + * It is equal to LocalAssignment.NONE whenever we are not in a group. */ - private final Map> currentTargetAssignment; + private LocalAssignment currentTargetAssignment; /** * If there is a reconciliation running (triggering commit, callbacks) for the @@ -222,11 +228,11 @@ public class MembershipManagerImpl implements MembershipManager { private boolean reconciliationInProgress; /** - * Epoch the member had when the reconciliation in progress started. This is used to identify if - * the member has rejoined while it was reconciling an assignment (in which case the result - * of the reconciliation is not applied.) + * True if a reconciliation is in progress and the member rejoined the group since the start + * of the reconciliation. Used to know that the reconciliation in progress should be + * interrupted and not be applied. */ - private int memberEpochOnReconciliationStart; + private boolean rejoinedWhileReconciliationInProgress; /** * If the member is currently leaving the group after a call to {@link #leaveGroup()}}, this @@ -253,7 +259,7 @@ public class MembershipManagerImpl implements MembershipManager { /** * Serves as the conduit by which we can report events to the application thread. This is needed as we send * {@link ConsumerRebalanceListenerCallbackNeededEvent callbacks} and, if needed, - * {@link ErrorBackgroundEvent errors} to the application thread. + * {@link ErrorEvent errors} to the application thread. */ private final BackgroundEventHandler backgroundEventHandler; @@ -263,6 +269,11 @@ public class MembershipManagerImpl implements MembershipManager { * when the timer is reset, only when it completes releasing its assignment. */ private CompletableFuture staleMemberAssignmentRelease; + + /* + * Measures successful rebalance latency and number of failed rebalances. + */ + private final RebalanceMetricsManager metricsManager; private final Time time; @@ -284,7 +295,35 @@ public MembershipManagerImpl(String groupId, LogContext logContext, Optional clientTelemetryReporter, BackgroundEventHandler backgroundEventHandler, - Time time) { + Time time, + Metrics metrics) { + this(groupId, + groupInstanceId, + rebalanceTimeoutMs, + serverAssignor, + subscriptions, + commitRequestManager, + metadata, + logContext, + clientTelemetryReporter, + backgroundEventHandler, + time, + new RebalanceMetricsManager(metrics)); + } + + // Visible for testing + MembershipManagerImpl(String groupId, + Optional groupInstanceId, + int rebalanceTimeoutMs, + Optional serverAssignor, + SubscriptionState subscriptions, + CommitRequestManager commitRequestManager, + ConsumerMetadata metadata, + LogContext logContext, + Optional clientTelemetryReporter, + BackgroundEventHandler backgroundEventHandler, + Time time, + RebalanceMetricsManager metricsManager) { this.groupId = groupId; this.state = MemberState.UNSUBSCRIBED; this.serverAssignor = serverAssignor; @@ -293,14 +332,15 @@ public MembershipManagerImpl(String groupId, this.commitRequestManager = commitRequestManager; this.metadata = metadata; this.assignedTopicNamesCache = new HashMap<>(); - this.currentTargetAssignment = new HashMap<>(); - this.currentAssignment = new HashMap<>(); + this.currentTargetAssignment = LocalAssignment.NONE; + this.currentAssignment = LocalAssignment.NONE; this.log = logContext.logger(MembershipManagerImpl.class); this.stateUpdatesListeners = new ArrayList<>(); this.clientTelemetryReporter = clientTelemetryReporter; this.rebalanceTimeoutMs = rebalanceTimeoutMs; this.backgroundEventHandler = backgroundEventHandler; this.time = time; + this.metricsManager = metricsManager; } /** @@ -314,10 +354,27 @@ private void transitionTo(MemberState nextState) { throw new IllegalStateException(String.format("Invalid state transition from %s to %s", state, nextState)); } + + if (isCompletingRebalance(state, nextState)) { + metricsManager.recordRebalanceEnded(time.milliseconds()); + } + if (isStartingRebalance(state, nextState)) { + metricsManager.recordRebalanceStarted(time.milliseconds()); + } + log.trace("Member {} with epoch {} transitioned from {} to {}.", memberId, memberEpoch, state, nextState); this.state = nextState; } + private static boolean isCompletingRebalance(MemberState currentState, MemberState nextState) { + return currentState == MemberState.RECONCILING && + (nextState == MemberState.STABLE || nextState == MemberState.ACKNOWLEDGING); + } + + private static boolean isStartingRebalance(MemberState currentState, MemberState nextState) { + return currentState != MemberState.RECONCILING && nextState == MemberState.RECONCILING; + } + /** * {@inheritDoc} */ @@ -354,7 +411,7 @@ public int memberEpoch() { * {@inheritDoc} */ @Override - public void onHeartbeatResponseReceived(ConsumerGroupHeartbeatResponseData response) { + public void onHeartbeatSuccess(ConsumerGroupHeartbeatResponseData response) { if (response.errorCode() != Errors.NONE.code()) { String errorMessage = String.format( "Unexpected error in Heartbeat response. Expected no error, but received: %s", @@ -397,12 +454,14 @@ public void onHeartbeatResponseReceived(ConsumerGroupHeartbeatResponseData respo return; } processAssignmentReceived(assignment); - - } else if (targetAssignmentReconciled()) { - transitionTo(MemberState.STABLE); } } + @Override + public void onHeartbeatFailure() { + metricsManager.maybeRecordRebalanceFailed(); + } + /** * @return True if the consumer is not a member of the group. */ @@ -449,9 +508,11 @@ private void processAssignmentReceived(ConsumerGroupHeartbeatResponseData.Assign */ private void replaceTargetAssignmentWithNewAssignment( ConsumerGroupHeartbeatResponseData.Assignment assignment) { - currentTargetAssignment.clear(); - assignment.topicPartitions().forEach(topicPartitions -> - currentTargetAssignment.put(topicPartitions.topicId(), new TreeSet<>(topicPartitions.partitions()))); + currentTargetAssignment.updateWith(assignment).ifPresent(updatedAssignment -> { + log.debug("Target assignment updated from {} to {}. Member will reconcile it on the next poll.", + currentTargetAssignment, updatedAssignment); + currentTargetAssignment = updatedAssignment; + }); } /** @@ -492,7 +553,7 @@ public void transitionToFenced() { log.error("onPartitionsLost callback invocation failed while releasing assignment" + " after member got fenced. Member will rejoin the group anyways.", error); } - updateSubscription(new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR), true); + clearSubscription(); if (state == MemberState.FENCED) { transitionToJoining(); } else { @@ -509,7 +570,11 @@ public void transitionToFenced() { public void transitionToFatal() { MemberState previousState = state; transitionTo(MemberState.FATAL); - log.error("Member {} with epoch {} transitioned to {} state", memberId, memberEpoch, MemberState.FATAL); + if (memberId.isEmpty()) { + log.error("Member {} with epoch {} transitioned to {} state", memberId, memberEpoch, MemberState.FATAL); + } else { + log.error("Non-member transitioned to {} state", MemberState.FATAL); + } notifyEpochChange(Optional.empty(), Optional.empty()); if (previousState == MemberState.UNSUBSCRIBED) { @@ -525,7 +590,7 @@ public void transitionToFatal() { log.error("onPartitionsLost callback invocation failed while releasing assignment" + "after member failed with fatal error.", error); } - updateSubscription(new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR), true); + clearSubscription(); }); } @@ -539,16 +604,14 @@ public void onSubscriptionUpdated() { } /** - * Update a new assignment by setting the assigned partitions in the member subscription. - * - * @param assignedPartitions Topic partitions to take as the new subscription assignment - * @param clearAssignments True if the pending assignments and metadata cache should be cleared + * Clear the assigned partitions in the member subscription, pending assignments and metadata cache. */ - private void updateSubscription(SortedSet assignedPartitions, - boolean clearAssignments) { - Collection assignedTopicPartitions = toTopicPartitionSet(assignedPartitions); - subscriptions.assignFromSubscribed(assignedTopicPartitions); - updateAssignmentLocally(assignedPartitions, clearAssignments); + private void clearSubscription() { + if (subscriptions.hasAutoAssignedPartitions()) { + subscriptions.assignFromSubscribed(Collections.emptySet()); + } + currentAssignment = LocalAssignment.NONE; + clearPendingAssignmentsAndLocalNamesCache(); } /** @@ -563,18 +626,6 @@ private void updateSubscriptionAwaitingCallback(SortedSet assi SortedSet addedPartitions) { Collection assignedTopicPartitions = toTopicPartitionSet(assignedPartitions); subscriptions.assignFromSubscribedAwaitingCallback(assignedTopicPartitions, addedPartitions); - updateAssignmentLocally(assignedPartitions, false); - } - - /** - * Make assignment effective on the group manager. - */ - private void updateAssignmentLocally(SortedSet assignedPartitions, - boolean clearAssignments) { - updateCurrentAssignment(assignedPartitions); - if (clearAssignments) { - clearPendingAssignmentsAndLocalNamesCache(); - } } /** @@ -590,6 +641,9 @@ public void transitionToJoining() { "the member is in FATAL state"); return; } + if (reconciliationInProgress) { + rejoinedWhileReconciliationInProgress = true; + } resetEpoch(); transitionTo(MemberState.JOINING); clearPendingAssignmentsAndLocalNamesCache(); @@ -602,7 +656,7 @@ public void transitionToJoining() { public CompletableFuture leaveGroup() { if (isNotInGroup()) { if (state == MemberState.FENCED) { - updateSubscription(new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR), true); + clearSubscription(); transitionTo(MemberState.UNSUBSCRIBED); } return CompletableFuture.completedFuture(null); @@ -621,7 +675,7 @@ public CompletableFuture leaveGroup() { CompletableFuture callbackResult = invokeOnPartitionsRevokedOrLostToReleaseAssignment(); callbackResult.whenComplete((result, error) -> { // Clear the subscription, no matter if the callback execution failed or succeeded. - updateSubscription(new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR), true); + clearSubscription(); // Transition to ensure that a heartbeat request is sent out to effectively leave the // group (even in the case where the member had no assignment to release or when the @@ -702,7 +756,7 @@ public void transitionToSendingLeaveGroup(boolean dueToExpiredPollTimer) { ConsumerGroupHeartbeatRequest.LEAVE_GROUP_STATIC_MEMBER_EPOCH : ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH; updateMemberEpoch(leaveEpoch); - currentAssignment = new HashMap<>(); + currentAssignment = LocalAssignment.NONE; transitionTo(MemberState.LEAVING); } @@ -822,7 +876,7 @@ private void transitionToStale() { log.error("onPartitionsLost callback invocation failed while releasing assignment" + " after member left group due to expired poll timer.", error); } - updateSubscription(new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR), true); + clearSubscription(); log.debug("Member {} sent leave group heartbeat and released its assignment. It will remain " + "in {} state until the poll timer is reset, and it will then rejoin the group", memberId, MemberState.STALE); @@ -843,12 +897,12 @@ private void transitionToStale() { */ void maybeReconcile() { if (targetAssignmentReconciled()) { - log.debug("Ignoring reconciliation attempt. Target assignment is equal to the " + + log.trace("Ignoring reconciliation attempt. Target assignment is equal to the " + "current assignment."); return; } if (reconciliationInProgress) { - log.debug("Ignoring reconciliation attempt. Another reconciliation is already in progress. Assignment " + + log.trace("Ignoring reconciliation attempt. Another reconciliation is already in progress. Assignment " + currentTargetAssignment + " will be handled in the next reconciliation loop."); return; } @@ -856,29 +910,25 @@ void maybeReconcile() { // Find the subset of the target assignment that can be resolved to topic names, and trigger a metadata update // if some topic IDs are not resolvable. SortedSet assignedTopicIdPartitions = findResolvableAssignmentAndTriggerMetadataUpdate(); + final LocalAssignment resolvedAssignment = new LocalAssignment(currentTargetAssignment.localEpoch, assignedTopicIdPartitions); + + if (!currentAssignment.isNone() && resolvedAssignment.partitions.equals(currentAssignment.partitions)) { + log.debug("There are unresolved partitions, and the resolvable fragment of the target assignment {} is equal to the current " + + "assignment. Bumping the local epoch of the assignment and acknowledging the partially resolved assignment", + resolvedAssignment.partitions); + currentAssignment = resolvedAssignment; + transitionTo(MemberState.ACKNOWLEDGING); + return; + } - SortedSet ownedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); - ownedPartitions.addAll(subscriptions.assignedPartitions()); + markReconciliationInProgress(); // Keep copy of assigned TopicPartitions created from the TopicIdPartitions that are // being reconciled. Needed for interactions with the centralized subscription state that // does not support topic IDs yet, and for the callbacks. SortedSet assignedTopicPartitions = toTopicPartitionSet(assignedTopicIdPartitions); - - // Check same assignment. Based on topic names for now, until topic IDs are properly - // supported in the centralized subscription state object. Note that this check is - // required to make sure that reconciliation is not triggered if the assignment ready to - // be reconciled is the same as the current one (even though the member may remain - // in RECONCILING state if it has some unresolved assignments). - boolean sameAssignmentReceived = assignedTopicPartitions.equals(ownedPartitions); - - if (sameAssignmentReceived) { - log.debug("Ignoring reconciliation attempt. Target assignment ready to reconcile {} " + - "is equal to the member current assignment {}.", assignedTopicPartitions, ownedPartitions); - return; - } - - markReconciliationInProgress(); + SortedSet ownedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); + ownedPartitions.addAll(subscriptions.assignedPartitions()); // Partitions to assign (not previously owned) SortedSet addedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); @@ -890,12 +940,13 @@ void maybeReconcile() { revokedPartitions.addAll(ownedPartitions); revokedPartitions.removeAll(assignedTopicPartitions); - log.info("Updating assignment with\n" + + log.info("Updating assignment with local epoch {}\n" + "\tAssigned partitions: {}\n" + "\tCurrent owned partitions: {}\n" + "\tAdded partitions (assigned - owned): {}\n" + "\tRevoked partitions (owned - assigned): {}\n", - assignedTopicIdPartitions, + resolvedAssignment.localEpoch, + assignedTopicPartitions, ownedPartitions, addedPartitions, revokedPartitions @@ -910,7 +961,7 @@ void maybeReconcile() { // best effort to commit the offsets in the case where the epoch might have changed while // the current reconciliation is in process. Note this is using the rebalance timeout as // it is the limit enforced by the broker to complete the reconciliation process. - commitResult = commitRequestManager.maybeAutoCommitSyncNow(getExpirationTimeForTimeout(rebalanceTimeoutMs)); + commitResult = commitRequestManager.maybeAutoCommitSyncBeforeRevocation(getExpirationTimeForTimeout(rebalanceTimeoutMs)); // Execute commit -> onPartitionsRevoked -> onPartitionsAssigned. commitResult.whenComplete((__, commitReqError) -> { @@ -924,7 +975,15 @@ void maybeReconcile() { log.debug("Auto-commit before reconciling new assignment completed successfully."); } - revokeAndAssign(assignedTopicIdPartitions, revokedPartitions, addedPartitions); + if (!maybeAbortReconciliation()) { + revokeAndAssign(resolvedAssignment, assignedTopicIdPartitions, revokedPartitions, addedPartitions); + } + + }).exceptionally(error -> { + if (error != null) { + log.error("Reconciliation failed.", error); + } + return null; }); } @@ -942,7 +1001,8 @@ long getExpirationTimeForTimeout(final long timeoutMs) { * then complete the reconciliation by updating the assignment and making the appropriate state * transition. Note that if any of the 2 callbacks fails, the reconciliation should fail. */ - private void revokeAndAssign(SortedSet assignedTopicIdPartitions, + private void revokeAndAssign(LocalAssignment resolvedAssignment, + SortedSet assignedTopicIdPartitions, SortedSet revokedPartitions, SortedSet addedPartitions) { CompletableFuture revocationResult; @@ -956,55 +1016,57 @@ private void revokeAndAssign(SortedSet assignedTopicIdPartitio // and assignment, executed sequentially). CompletableFuture reconciliationResult = revocationResult.thenCompose(__ -> { - boolean memberHasRejoined = memberEpochOnReconciliationStart != memberEpoch; - if (state == MemberState.RECONCILING && !memberHasRejoined) { + if (!maybeAbortReconciliation()) { // Apply assignment return assignPartitions(assignedTopicIdPartitions, addedPartitions); - } else { - log.debug("Revocation callback completed but the member already " + - "transitioned out of the reconciling state for epoch {} into " + - "{} state with epoch {}. Interrupting reconciliation as it's " + - "not relevant anymore,", memberEpochOnReconciliationStart, state, memberEpoch); - String reason = interruptedReconciliationErrorMessage(); - CompletableFuture res = new CompletableFuture<>(); - res.completeExceptionally(new KafkaException("Interrupting reconciliation" + - " after revocation. " + reason)); - return res; } + return CompletableFuture.completedFuture(null); }); - reconciliationResult.whenComplete((result, error) -> { - markReconciliationCompleted(); + reconciliationResult.whenComplete((__, error) -> { if (error != null) { // Leaving member in RECONCILING state after callbacks fail. The member // won't send the ack, and the expectation is that the broker will kick the // member out of the group after the rebalance timeout expires, leading to a // RECONCILING -> FENCED transition. log.error("Reconciliation failed.", error); + markReconciliationCompleted(); } else { - boolean memberHasRejoined = memberEpochOnReconciliationStart != memberEpoch; - if (state == MemberState.RECONCILING && !memberHasRejoined) { + if (reconciliationInProgress && !maybeAbortReconciliation()) { + currentAssignment = resolvedAssignment; + // Reschedule the auto commit starting from now that the member has a new assignment. commitRequestManager.resetAutoCommitTimer(); // Make assignment effective on the broker by transitioning to send acknowledge. transitionTo(MemberState.ACKNOWLEDGING); - } else { - String reason = interruptedReconciliationErrorMessage(); - log.error("Interrupting reconciliation after partitions assigned callback " + - "completed. " + reason); + markReconciliationCompleted(); } } }); } + /** + * @return True if the reconciliation in progress should not continue. This could be because + * the member is not in RECONCILING state anymore (member failed or is leaving the group), or + * if it has rejoined the group (note that after rejoining the member could be RECONCILING + * again, so checking the state is not enough) + */ + boolean maybeAbortReconciliation() { + boolean shouldAbort = state != MemberState.RECONCILING || rejoinedWhileReconciliationInProgress; + if (shouldAbort) { + String reason = rejoinedWhileReconciliationInProgress ? + "the member has re-joined the group" : + "the member already transitioned out of the reconciling state into " + state; + log.info("Interrupting reconciliation that is not relevant anymore because " + reason); + markReconciliationCompleted(); + } + return shouldAbort; + } + // Visible for testing. - void updateCurrentAssignment(Set assignedTopicIdPartitions) { - currentAssignment.clear(); - assignedTopicIdPartitions.forEach(topicIdPartition -> { - Uuid topicId = topicIdPartition.topicId(); - currentAssignment.computeIfAbsent(topicId, k -> new TreeSet<>()).add(topicIdPartition.partition()); - }); + void updateAssignment(Map> partitions) { + currentAssignment = new LocalAssignment(0, partitions); } /** @@ -1016,25 +1078,12 @@ private SortedSet toTopicPartitionSet(SortedSet findResolvableAssignmentAndTriggerMetadataUpdate() { final SortedSet assignmentReadyToReconcile = new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR); - final HashMap> unresolved = new HashMap<>(currentTargetAssignment); + final HashMap> unresolved = new HashMap<>(currentTargetAssignment.partitions); // Try to resolve topic names from metadata cache or subscription cache, and move // assignments from the "unresolved" collection, to the "assignmentReadyToReconcile" one. @@ -1336,7 +1386,7 @@ private void logPausedPartitionsBeingRevoked(Set partitionsToRev * or the next reconciliation loop). Remove all elements from the topic names cache. */ private void clearPendingAssignmentsAndLocalNamesCache() { - currentTargetAssignment.clear(); + currentTargetAssignment = LocalAssignment.NONE; assignedTopicNamesCache.clear(); } @@ -1378,7 +1428,7 @@ public Optional serverAssignor() { * {@inheritDoc} */ @Override - public Map> currentAssignment() { + public LocalAssignment currentAssignment() { return this.currentAssignment; } @@ -1403,9 +1453,15 @@ Set topicsAwaitingReconciliation() { * Visible for testing. */ Map> topicPartitionsAwaitingReconciliation() { + if (currentTargetAssignment == LocalAssignment.NONE) { + return Collections.emptyMap(); + } + if (currentAssignment == LocalAssignment.NONE) { + return currentTargetAssignment.partitions; + } final Map> topicPartitionMap = new HashMap<>(); - currentTargetAssignment.forEach((topicId, targetPartitions) -> { - final SortedSet reconciledPartitions = currentAssignment.get(topicId); + currentTargetAssignment.partitions.forEach((topicId, targetPartitions) -> { + final SortedSet reconciledPartitions = currentAssignment.partitions.get(topicId); if (!targetPartitions.equals(reconciledPartitions)) { final TreeSet missingPartitions = new TreeSet<>(targetPartitions); if (reconciledPartitions != null) { @@ -1446,4 +1502,10 @@ public PollResult poll(final long currentTimeMs) { } return PollResult.EMPTY; } + + // visible for testing + List stateListeners() { + return unmodifiableList(stateUpdatesListeners); + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java index db7770cbda..47a5df6d1d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetCommitCallbackInvoker.java @@ -19,7 +19,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.FencedInstanceIdException; import java.util.Map; import java.util.concurrent.BlockingQueue; @@ -33,7 +32,6 @@ */ public class OffsetCommitCallbackInvoker { private final ConsumerInterceptors interceptors; - private boolean hasFencedException = false; OffsetCommitCallbackInvoker(ConsumerInterceptors interceptors) { this.interceptors = interceptors; @@ -62,19 +60,11 @@ public void executeCallbacks() { while (!callbackQueue.isEmpty()) { OffsetCommitCallbackTask task = callbackQueue.poll(); if (task != null) { - - if (task.exception instanceof FencedInstanceIdException) - hasFencedException = true; - task.callback.onComplete(task.offsets, task.exception); } } } - public boolean hasFencedException() { - return hasFencedException; - } - private static class OffsetCommitCallbackTask { public final Map offsets; public final Exception exception; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java index 34f4b30c44..c5156e9e0b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java @@ -25,7 +25,7 @@ import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetData; import org.apache.kafka.clients.consumer.internals.OffsetFetcherUtils.ListOffsetResult; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.common.ClusterResource; import org.apache.kafka.common.ClusterResourceListener; import org.apache.kafka.common.IsolationLevel; @@ -199,7 +199,7 @@ public CompletableFuture resetPositionsIfNeeded() { try { offsetResetTimestamps = offsetFetcherUtils.getOffsetResetTimestamp(); } catch (Exception e) { - backgroundEventHandler.add(new ErrorBackgroundEvent(e)); + backgroundEventHandler.add(new ErrorEvent(e)); return CompletableFuture.completedFuture(null); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java index 2d90a3ad70..75d87432db 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java @@ -123,7 +123,8 @@ public static Supplier supplier(final Time time, final Supplier networkClientDelegateSupplier, final Optional clientTelemetryReporter, final Metrics metrics, - final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker + final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker, + final MemberStateListener applicationThreadMemberStateListener ) { return new CachedSupplier() { @Override @@ -189,8 +190,10 @@ protected RequestManagers create() { logContext, clientTelemetryReporter, backgroundEventHandler, - time); + time, + metrics); membershipManager.registerStateListener(commit); + membershipManager.registerStateListener(applicationThreadMemberStateListener); heartbeatRequestManager = new HeartbeatRequestManager( logContext, time, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestState.java index 321be5e8fe..a888e7831a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestState.java @@ -102,7 +102,7 @@ public boolean requestInFlight() { } public void onSendAttempt(final long currentTimeMs) { - // Here we update the timer everytime we try to send a request. Also increment number of attempts. + // Here we update the timer everytime we try to send a request. this.lastSentMs = currentTimeMs; } 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 d466a4a1b2..87d471d2e1 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 @@ -540,6 +540,14 @@ public synchronized FetchPosition position(TopicPartition tp) { return assignedState(tp).position; } + public synchronized FetchPosition positionOrNull(TopicPartition tp) { + final TopicPartitionState state = assignedStateOrNull(tp); + if (state == null) { + return null; + } + return assignedState(tp).position; + } + public synchronized Long partitionLag(TopicPartition tp, IsolationLevel isolationLevel) { TopicPartitionState topicPartitionState = assignedState(tp); if (topicPartitionState.position == null) { @@ -579,14 +587,37 @@ synchronized void updateHighWatermark(TopicPartition tp, long highWatermark) { assignedState(tp).highWatermark(highWatermark); } - synchronized void updateLogStartOffset(TopicPartition tp, long logStartOffset) { - assignedState(tp).logStartOffset(logStartOffset); + synchronized boolean tryUpdatingHighWatermark(TopicPartition tp, long highWatermark) { + final TopicPartitionState state = assignedStateOrNull(tp); + if (state != null) { + assignedState(tp).highWatermark(highWatermark); + return true; + } + return false; + } + + synchronized boolean tryUpdatingLogStartOffset(TopicPartition tp, long highWatermark) { + final TopicPartitionState state = assignedStateOrNull(tp); + if (state != null) { + assignedState(tp).logStartOffset(highWatermark); + return true; + } + return false; } synchronized void updateLastStableOffset(TopicPartition tp, long lastStableOffset) { assignedState(tp).lastStableOffset(lastStableOffset); } + synchronized boolean tryUpdatingLastStableOffset(TopicPartition tp, long lastStableOffset) { + final TopicPartitionState state = assignedStateOrNull(tp); + if (state != null) { + assignedState(tp).lastStableOffset(lastStableOffset); + return true; + } + return false; + } + /** * Set the preferred read replica with a lease timeout. After this time, the replica will no longer be valid and * {@link #preferredReadReplica(TopicPartition, long)} will return an empty result. @@ -599,6 +630,28 @@ public synchronized void updatePreferredReadReplica(TopicPartition tp, int prefe assignedState(tp).updatePreferredReadReplica(preferredReadReplicaId, timeMs); } + /** + * Tries to set the preferred read replica with a lease timeout. After this time, the replica will no longer be valid and + * {@link #preferredReadReplica(TopicPartition, long)} will return an empty result. If the preferred replica of + * the partition could not be updated (e.g. because the partition is not assigned) this method will return + * {@code false}, otherwise it will return {@code true}. + * + * @param tp The topic partition + * @param preferredReadReplicaId The preferred read replica + * @param timeMs The time at which this preferred replica is no longer valid + * @return {@code true} if the preferred read replica was updated, {@code false} otherwise. + */ + public synchronized boolean tryUpdatingPreferredReadReplica(TopicPartition tp, + int preferredReadReplicaId, + LongSupplier timeMs) { + final TopicPartitionState state = assignedStateOrNull(tp); + if (state != null) { + assignedState(tp).updatePreferredReadReplica(preferredReadReplicaId, timeMs); + return true; + } + return false; + } + /** * Get the preferred read replica * @@ -655,6 +708,14 @@ public void requestOffsetReset(TopicPartition partition) { requestOffsetReset(partition, defaultResetStrategy); } + public synchronized void requestOffsetResetIfPartitionAssigned(TopicPartition partition) { + final TopicPartitionState state = assignedStateOrNull(partition); + if (state != null) { + state.reset(defaultResetStrategy); + } + } + + synchronized void setNextAllowedRetry(Set partitions, long nextAllowResetTimeMs) { for (TopicPartition partition : partitions) { assignedState(partition).setNextAllowedRetry(nextAllowResetTimeMs); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java index 5a030f6307..209d5e41be 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/WakeupTrigger.java @@ -73,6 +73,8 @@ public CompletableFuture setActiveTask(final CompletableFuture current } else if (task instanceof WakeupFuture) { currentTask.completeExceptionally(new WakeupException()); return null; + } else if (task instanceof DisabledWakeups) { + return task; } // last active state is still active throw new KafkaException("Last active task is still active"); @@ -88,6 +90,8 @@ public void setFetchAction(final FetchBuffer fetchBuffer) { } else if (task instanceof WakeupFuture) { throwWakeupException.set(true); return null; + } else if (task instanceof DisabledWakeups) { + return task; } // last active state is still active throw new IllegalStateException("Last active task is still active"); @@ -97,6 +101,10 @@ public void setFetchAction(final FetchBuffer fetchBuffer) { } } + public void disableWakeups() { + pendingTask.set(new DisabledWakeups()); + } + public void clearTask() { pendingTask.getAndUpdate(task -> { if (task == null) { @@ -131,6 +139,9 @@ Wakeupable getPendingTask() { interface Wakeupable { } + // Set to block wakeups from happening and pending actions to be registered. + static class DisabledWakeups implements Wakeupable { } + static class ActiveFuture implements Wakeupable { private final CompletableFuture future; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java new file mode 100644 index 0000000000..3347002cc6 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.utils.Timer; + +import java.util.List; +import java.util.Map; + +public abstract class AbstractTopicMetadataEvent extends CompletableApplicationEvent>> { + + protected AbstractTopicMetadataEvent(final Type type, final Timer timer) { + super(type, timer); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java similarity index 73% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java index ee0b6ffa61..bda18e6421 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java @@ -16,15 +16,11 @@ */ package org.apache.kafka.clients.consumer.internals.events; -public class LeaveOnCloseApplicationEvent extends CompletableApplicationEvent { - public LeaveOnCloseApplicationEvent() { - super(Type.LEAVE_ON_CLOSE); - } +import org.apache.kafka.common.utils.Timer; + +public class AllTopicsMetadataEvent extends AbstractTopicMetadataEvent { - @Override - public String toString() { - return "LeaveOnCloseApplicationEvent{" + - toStringBase() + - '}'; + public AllTopicsMetadataEvent(final Timer timer) { + super(Type.ALL_TOPICS_METADATA, timer); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java index ac7ccc56c5..2897117da8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java @@ -16,52 +16,65 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer; +import org.apache.kafka.common.Uuid; + import java.util.Objects; /** - * This is the abstract definition of the events created by the KafkaConsumer API + * This is the abstract definition of the events created by the {@link AsyncKafkaConsumer} on the user's + * application thread. */ public abstract class ApplicationEvent { public enum Type { COMMIT_ASYNC, COMMIT_SYNC, POLL, FETCH_COMMITTED_OFFSETS, NEW_TOPICS_METADATA_UPDATE, ASSIGNMENT_CHANGE, - LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, SUBSCRIPTION_CHANGE, + LIST_OFFSETS, RESET_POSITIONS, VALIDATE_POSITIONS, TOPIC_METADATA, ALL_TOPICS_METADATA, SUBSCRIPTION_CHANGE, UNSUBSCRIBE, CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED, COMMIT_ON_CLOSE, LEAVE_ON_CLOSE } private final Type type; + /** + * This identifies a particular event. It is used to disambiguate events via {@link #hashCode()} and + * {@link #equals(Object)} and can be used in log messages when debugging. + */ + private final Uuid id; + protected ApplicationEvent(Type type) { this.type = Objects.requireNonNull(type); + this.id = Uuid.randomUuid(); } public Type type() { return type; } + public Uuid id() { + return id; + } + @Override - public boolean equals(Object o) { + public final boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - ApplicationEvent that = (ApplicationEvent) o; - - return type == that.type; + return type == that.type && id.equals(that.id); } @Override - public int hashCode() { - return type.hashCode(); + public final int hashCode() { + return Objects.hash(type, id); } protected String toStringBase() { - return "type=" + type; + return "type=" + type + ", id=" + id; } @Override - public String toString() { - return "ApplicationEvent{" + + public final String toString() { + return getClass().getSimpleName() + "{" + toStringBase() + '}'; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java index 7535edf597..eac1cc3d62 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.common.internals.IdempotentCloser; @@ -99,10 +100,9 @@ public long maximumTimeToWait() { * *

* - * See {@link CompletableApplicationEvent#get(Timer)} and {@link Future#get(long, TimeUnit)} for more details. + * See {@link ConsumerUtils#getResult(Future, Timer)} and {@link Future#get(long, TimeUnit)} for more details. * * @param event A {@link CompletableApplicationEvent} created by the polling thread - * @param timer Timer for which to wait for the event to complete * @return Value that is the result of the event * @param Type of return value of the event */ @@ -110,7 +110,7 @@ public T addAndGet(final CompletableApplicationEvent event, final Timer t Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null"); Objects.requireNonNull(timer, "Timer provided to addAndGet must be non-null"); add(event); - return event.get(timer); + return ConsumerUtils.getResult(event.future(), timer); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index 9e48b4de6d..3382530746 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.internals.CachedSupplier; import org.apache.kafka.clients.consumer.internals.CommitRequestManager; @@ -34,6 +35,7 @@ import java.util.Objects; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.function.BiConsumer; import java.util.function.Supplier; /** @@ -65,23 +67,24 @@ public boolean process() { return process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); } + @SuppressWarnings({"CyclomaticComplexity"}) @Override public void process(ApplicationEvent event) { switch (event.type()) { case COMMIT_ASYNC: - process((AsyncCommitApplicationEvent) event); + process((AsyncCommitEvent) event); return; case COMMIT_SYNC: - process((SyncCommitApplicationEvent) event); + process((SyncCommitEvent) event); return; case POLL: - process((PollApplicationEvent) event); + process((PollEvent) event); return; case FETCH_COMMITTED_OFFSETS: - process((FetchCommittedOffsetsApplicationEvent) event); + process((FetchCommittedOffsetsEvent) event); return; case NEW_TOPICS_METADATA_UPDATE: @@ -89,31 +92,35 @@ public void process(ApplicationEvent event) { return; case ASSIGNMENT_CHANGE: - process((AssignmentChangeApplicationEvent) event); + process((AssignmentChangeEvent) event); return; case TOPIC_METADATA: - process((TopicMetadataApplicationEvent) event); + process((TopicMetadataEvent) event); + return; + + case ALL_TOPICS_METADATA: + process((AllTopicsMetadataEvent) event); return; case LIST_OFFSETS: - process((ListOffsetsApplicationEvent) event); + process((ListOffsetsEvent) event); return; case RESET_POSITIONS: - process((ResetPositionsApplicationEvent) event); + process((ResetPositionsEvent) event); return; case VALIDATE_POSITIONS: - process((ValidatePositionsApplicationEvent) event); + process((ValidatePositionsEvent) event); return; case SUBSCRIPTION_CHANGE: - process((SubscriptionChangeApplicationEvent) event); + process((SubscriptionChangeEvent) event); return; case UNSUBSCRIBE: - process((UnsubscribeApplicationEvent) event); + process((UnsubscribeEvent) event); return; case CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED: @@ -121,11 +128,11 @@ public void process(ApplicationEvent event) { return; case COMMIT_ON_CLOSE: - process((CommitOnCloseApplicationEvent) event); + process((CommitOnCloseEvent) event); return; case LEAVE_ON_CLOSE: - process((LeaveOnCloseApplicationEvent) event); + process((LeaveOnCloseEvent) event); return; default: @@ -133,7 +140,7 @@ public void process(ApplicationEvent event) { } } - private void process(final PollApplicationEvent event) { + private void process(final PollEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { return; } @@ -142,40 +149,46 @@ private void process(final PollApplicationEvent event) { requestManagers.heartbeatRequestManager.ifPresent(hrm -> hrm.resetPollTimer(event.pollTimeMs())); } - private void process(final AsyncCommitApplicationEvent event) { + private void process(final AsyncCommitEvent event) { + if (!requestManagers.commitRequestManager.isPresent()) { + return; + } + CommitRequestManager manager = requestManagers.commitRequestManager.get(); - CompletableFuture commitResult = manager.commitAsync(event.offsets()); - event.chain(commitResult); + CompletableFuture future = manager.commitAsync(event.offsets()); + future.whenComplete(complete(event.future())); } - private void process(final SyncCommitApplicationEvent event) { + private void process(final SyncCommitEvent event) { + if (!requestManagers.commitRequestManager.isPresent()) { + return; + } + CommitRequestManager manager = requestManagers.commitRequestManager.get(); - long expirationTimeoutMs = getExpirationTimeForTimeout(event.retryTimeoutMs()); - CompletableFuture commitResult = manager.commitSync(event.offsets(), expirationTimeoutMs); - event.chain(commitResult); + CompletableFuture future = manager.commitSync(event.offsets(), event.deadlineMs()); + future.whenComplete(complete(event.future())); } - private void process(final FetchCommittedOffsetsApplicationEvent event) { + private void process(final FetchCommittedOffsetsEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { event.future().completeExceptionally(new KafkaException("Unable to fetch committed " + "offset because the CommittedRequestManager is not available. Check if group.id was set correctly")); return; } CommitRequestManager manager = requestManagers.commitRequestManager.get(); - long expirationTimeMs = getExpirationTimeForTimeout(event.timeout()); - event.chain(manager.fetchOffsets(event.partitions(), expirationTimeMs)); + CompletableFuture> future = manager.fetchOffsets(event.partitions(), event.deadlineMs()); + future.whenComplete(complete(event.future())); } private void process(final NewTopicsMetadataUpdateRequestEvent ignored) { metadata.requestUpdateForNewTopics(); } - /** * Commit all consumed if auto-commit is enabled. Note this will trigger an async commit, * that will not be retried if the commit request fails. */ - private void process(final AssignmentChangeApplicationEvent event) { + private void process(final AssignmentChangeEvent event) { if (!requestManagers.commitRequestManager.isPresent()) { return; } @@ -184,11 +197,11 @@ private void process(final AssignmentChangeApplicationEvent event) { manager.maybeAutoCommitAsync(); } - private void process(final ListOffsetsApplicationEvent event) { + private void process(final ListOffsetsEvent event) { final CompletableFuture> future = requestManagers.offsetsRequestManager.fetchOffsets(event.timestampsToSearch(), event.requireTimestamps()); - event.chain(future); + future.whenComplete(complete(event.future())); } /** @@ -196,7 +209,7 @@ private void process(final ListOffsetsApplicationEvent event) { * consumer join the group if it is not part of it yet, or send the updated subscription if * it is already a member. */ - private void process(final SubscriptionChangeApplicationEvent ignored) { + private void process(final SubscriptionChangeEvent ignored) { if (!requestManagers.heartbeatRequestManager.isPresent()) { log.warn("Group membership manager not present when processing a subscribe event"); return; @@ -213,38 +226,37 @@ private void process(final SubscriptionChangeApplicationEvent ignored) { * execution for releasing the assignment completes, and the request to leave * the group is sent out. */ - private void process(final UnsubscribeApplicationEvent event) { + private void process(final UnsubscribeEvent event) { if (!requestManagers.heartbeatRequestManager.isPresent()) { KafkaException error = new KafkaException("Group membership manager not present when processing an unsubscribe event"); event.future().completeExceptionally(error); return; } MembershipManager membershipManager = requestManagers.heartbeatRequestManager.get().membershipManager(); - CompletableFuture result = membershipManager.leaveGroup(); - event.chain(result); + CompletableFuture future = membershipManager.leaveGroup(); + future.whenComplete(complete(event.future())); } - private void process(final ResetPositionsApplicationEvent event) { - CompletableFuture result = requestManagers.offsetsRequestManager.resetPositionsIfNeeded(); - event.chain(result); + private void process(final ResetPositionsEvent event) { + CompletableFuture future = requestManagers.offsetsRequestManager.resetPositionsIfNeeded(); + future.whenComplete(complete(event.future())); } - private void process(final ValidatePositionsApplicationEvent event) { - CompletableFuture result = requestManagers.offsetsRequestManager.validatePositionsIfNeeded(); - event.chain(result); + private void process(final ValidatePositionsEvent event) { + CompletableFuture future = requestManagers.offsetsRequestManager.validatePositionsIfNeeded(); + future.whenComplete(complete(event.future())); } - private void process(final TopicMetadataApplicationEvent event) { - final CompletableFuture>> future; - - long expirationTimeMs = getExpirationTimeForTimeout(event.getTimeoutMs()); - if (event.isAllTopics()) { - future = requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(expirationTimeMs); - } else { - future = requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), expirationTimeMs); - } + private void process(final TopicMetadataEvent event) { + final CompletableFuture>> future = + requestManagers.topicMetadataRequestManager.requestTopicMetadata(event.topic(), event.deadlineMs()); + future.whenComplete(complete(event.future())); + } - event.chain(future); + private void process(final AllTopicsMetadataEvent event) { + final CompletableFuture>> future = + requestManagers.topicMetadataRequestManager.requestAllTopicsMetadata(event.deadlineMs()); + future.whenComplete(complete(event.future())); } private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event) { @@ -259,14 +271,14 @@ private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event manager.consumerRebalanceListenerCallbackCompleted(event); } - private void process(final CommitOnCloseApplicationEvent event) { + private void process(final CommitOnCloseEvent event) { if (!requestManagers.commitRequestManager.isPresent()) return; log.debug("Signal CommitRequestManager closing"); requestManagers.commitRequestManager.get().signalClose(); } - private void process(final LeaveOnCloseApplicationEvent event) { + private void process(final LeaveOnCloseEvent event) { if (!requestManagers.heartbeatRequestManager.isPresent()) { event.future().complete(null); return; @@ -277,20 +289,16 @@ private void process(final LeaveOnCloseApplicationEvent event) { log.debug("Leaving group before closing"); CompletableFuture future = membershipManager.leaveGroup(); // The future will be completed on heartbeat sent - event.chain(future); + future.whenComplete(complete(event.future())); } - /** - * @return Expiration time in milliseconds calculated with the current time plus the given - * timeout. Returns Long.MAX_VALUE if the expiration overflows it. - * Visible for testing. - */ - long getExpirationTimeForTimeout(final long timeoutMs) { - long expiration = System.currentTimeMillis() + timeoutMs; - if (expiration < 0) { - return Long.MAX_VALUE; - } - return expiration; + private BiConsumer complete(final CompletableFuture b) { + return (value, exception) -> { + if (exception != null) + b.completeExceptionally(exception); + else + b.complete(value); + }; } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeEvent.java similarity index 56% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeEvent.java index ccf7199f26..c9efa2e9df 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AssignmentChangeEvent.java @@ -22,13 +22,12 @@ import java.util.Collections; import java.util.Map; -public class AssignmentChangeApplicationEvent extends ApplicationEvent { +public class AssignmentChangeEvent extends ApplicationEvent { private final Map offsets; private final long currentTimeMs; - public AssignmentChangeApplicationEvent(final Map offsets, - final long currentTimeMs) { + public AssignmentChangeEvent(final Map offsets, final long currentTimeMs) { super(Type.ASSIGNMENT_CHANGE); this.offsets = Collections.unmodifiableMap(offsets); this.currentTimeMs = currentTimeMs; @@ -43,31 +42,7 @@ public long currentTimeMs() { } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - AssignmentChangeApplicationEvent that = (AssignmentChangeApplicationEvent) o; - - if (currentTimeMs != that.currentTimeMs) return false; - return offsets.equals(that.offsets); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + offsets.hashCode(); - result = 31 * result + (int) (currentTimeMs ^ (currentTimeMs >>> 32)); - return result; - } - - @Override - public String toString() { - return "AssignmentChangeApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets + - ", currentTimeMs=" + currentTimeMs + - '}'; + protected String toStringBase() { + return super.toStringBase() + ", offsets=" + offsets + ", currentTimeMs=" + currentTimeMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java similarity index 73% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java index 7a939ce3cf..c36f0534b3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AsyncCommitEvent.java @@ -18,22 +18,15 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; + import java.util.Map; /** * Event to commit offsets without waiting for a response, so the request won't be retried. */ -public class AsyncCommitApplicationEvent extends CommitApplicationEvent { - - public AsyncCommitApplicationEvent(final Map offsets) { - super(offsets, Type.COMMIT_ASYNC); - } +public class AsyncCommitEvent extends CommitEvent { - @Override - public String toString() { - return "AsyncCommitApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets() + - '}'; + public AsyncCommitEvent(final Map offsets) { + super(Type.COMMIT_ASYNC, offsets, Long.MAX_VALUE); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java index e5d522201e..4241482bca 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; +import org.apache.kafka.common.Uuid; import java.util.Objects; @@ -26,41 +27,50 @@ public abstract class BackgroundEvent { public enum Type { - ERROR, CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, GROUP_METADATA_UPDATE + ERROR, CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED } private final Type type; - public BackgroundEvent(Type type) { + /** + * This identifies a particular event. It is used to disambiguate events via {@link #hashCode()} and + * {@link #equals(Object)} and can be used in log messages when debugging. + */ + private final Uuid id; + + protected BackgroundEvent(Type type) { this.type = Objects.requireNonNull(type); + this.id = Uuid.randomUuid(); } public Type type() { return type; } + public Uuid id() { + return id; + } + @Override - public boolean equals(Object o) { + public final boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - BackgroundEvent that = (BackgroundEvent) o; - - return type == that.type; + return type == that.type && id.equals(that.id); } @Override - public int hashCode() { - return type.hashCode(); + public final int hashCode() { + return Objects.hash(type, id); } protected String toStringBase() { - return "type=" + type; + return "type=" + type + ", id=" + id; } @Override - public String toString() { - return "BackgroundEvent{" + + public final String toString() { + return getClass().getSimpleName() + "{" + toStringBase() + '}'; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java index 103493d253..48421484f1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEventHandler.java @@ -26,7 +26,7 @@ /** * An event handler that receives {@link BackgroundEvent background events} from the * {@link ConsumerNetworkThread network thread} which are then made available to the application thread - * via the {@link BackgroundEventProcessor}. + * via an {@link EventProcessor}. */ public class BackgroundEventHandler { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java similarity index 61% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java index 69d969d7b0..1da7b84039 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java @@ -18,26 +18,40 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.Map; -public abstract class CommitApplicationEvent extends CompletableApplicationEvent { +public abstract class CommitEvent extends CompletableApplicationEvent { /** * Offsets to commit per partition. */ private final Map offsets; - public CommitApplicationEvent(final Map offsets, Type type) { - super(type); - this.offsets = Collections.unmodifiableMap(offsets); + protected CommitEvent(final Type type, final Map offsets, final Timer timer) { + super(type, timer); + this.offsets = validate(offsets); + } + + protected CommitEvent(final Type type, final Map offsets, final long deadlineMs) { + super(type, deadlineMs); + this.offsets = validate(offsets); + } + /** + * Validates the offsets are not negative and then returns the given offset map as + * {@link Collections#unmodifiableMap(Map) as unmodifiable}. + */ + private static Map validate(final Map offsets) { for (OffsetAndMetadata offsetAndMetadata : offsets.values()) { if (offsetAndMetadata.offset() < 0) { throw new IllegalArgumentException("Invalid offset: " + offsetAndMetadata.offset()); } } + + return Collections.unmodifiableMap(offsets); } public Map offsets() { @@ -45,20 +59,7 @@ public Map offsets() { } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - CommitApplicationEvent that = (CommitApplicationEvent) o; - - return offsets.equals(that.offsets); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + offsets.hashCode(); - return result; + protected String toStringBase() { + return super.toStringBase() + ", offsets=" + offsets; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseEvent.java similarity index 76% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseEvent.java index 4cc07e945f..7d2e29fced 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitOnCloseEvent.java @@ -16,16 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; -public class CommitOnCloseApplicationEvent extends ApplicationEvent { +public class CommitOnCloseEvent extends ApplicationEvent { - public CommitOnCloseApplicationEvent() { + public CommitOnCloseEvent() { super(Type.COMMIT_ON_CLOSE); } - - @Override - public String toString() { - return "CommitOnCloseApplicationEvent{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index 365c620e0c..dae9e9f101 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -16,9 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; import org.apache.kafka.common.utils.Timer; +import java.util.Objects; import java.util.concurrent.CompletableFuture; /** @@ -30,57 +30,32 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent implements CompletableEvent { private final CompletableFuture future; + private final long deadlineMs; - protected CompletableApplicationEvent(Type type) { + protected CompletableApplicationEvent(final Type type, final Timer timer) { super(type); this.future = new CompletableFuture<>(); + Objects.requireNonNull(timer); + this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); } - public CompletableFuture future() { - return future; - } - - public T get(Timer timer) { - return ConsumerUtils.getResult(future, timer); - } - - public void chain(final CompletableFuture providedFuture) { - providedFuture.whenComplete((value, exception) -> { - if (exception != null) { - this.future.completeExceptionally(exception); - } else { - this.future.complete(value); - } - }); + protected CompletableApplicationEvent(final Type type, final long deadlineMs) { + super(type); + this.future = new CompletableFuture<>(); + this.deadlineMs = deadlineMs; } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - CompletableApplicationEvent that = (CompletableApplicationEvent) o; - - return future.equals(that.future); + public CompletableFuture future() { + return future; } - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + future.hashCode(); - return result; + public long deadlineMs() { + return deadlineMs; } @Override protected String toStringBase() { - return super.toStringBase() + ", future=" + future; - } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; + return super.toStringBase() + ", future=" + future + ", deadlineMs=" + deadlineMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index 640ee6103a..1a58515a5c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -28,42 +28,18 @@ public abstract class CompletableBackgroundEvent extends BackgroundEvent impl private final CompletableFuture future; - protected CompletableBackgroundEvent(Type type) { + protected CompletableBackgroundEvent(final Type type) { super(type); this.future = new CompletableFuture<>(); } + @Override public CompletableFuture future() { return future; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - CompletableBackgroundEvent that = (CompletableBackgroundEvent) o; - - return future.equals(that.future); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + future.hashCode(); - return result; - } - @Override protected String toStringBase() { return super.toStringBase() + ", future=" + future; } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 8fdcc20fa8..97559d8cb9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -21,5 +21,4 @@ public interface CompletableEvent { CompletableFuture future(); - } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java index b260c6154e..a10e98df1d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackCompletedEvent.java @@ -34,9 +34,9 @@ public class ConsumerRebalanceListenerCallbackCompletedEvent extends Application private final CompletableFuture future; private final Optional error; - public ConsumerRebalanceListenerCallbackCompletedEvent(ConsumerRebalanceListenerMethodName methodName, - CompletableFuture future, - Optional error) { + public ConsumerRebalanceListenerCallbackCompletedEvent(final ConsumerRebalanceListenerMethodName methodName, + final CompletableFuture future, + final Optional error) { super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_COMPLETED); this.methodName = Objects.requireNonNull(methodName); this.future = Objects.requireNonNull(future); @@ -55,24 +55,6 @@ public Optional error() { return error; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - ConsumerRebalanceListenerCallbackCompletedEvent that = (ConsumerRebalanceListenerCallbackCompletedEvent) o; - - return methodName == that.methodName && - future.equals(that.future) && - error.equals(that.error); - } - - @Override - public int hashCode() { - return Objects.hash(methodName, future, error); - } - @Override protected String toStringBase() { return super.toStringBase() + @@ -80,11 +62,4 @@ protected String toStringBase() { ", future=" + future + ", error=" + error; } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java index 7b17c034ab..6ce833580c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java @@ -37,8 +37,8 @@ public class ConsumerRebalanceListenerCallbackNeededEvent extends CompletableBac private final ConsumerRebalanceListenerMethodName methodName; private final SortedSet partitions; - public ConsumerRebalanceListenerCallbackNeededEvent(ConsumerRebalanceListenerMethodName methodName, - SortedSet partitions) { + public ConsumerRebalanceListenerCallbackNeededEvent(final ConsumerRebalanceListenerMethodName methodName, + final SortedSet partitions) { super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED); this.methodName = Objects.requireNonNull(methodName); this.partitions = Collections.unmodifiableSortedSet(partitions); @@ -52,36 +52,10 @@ public SortedSet partitions() { return partitions; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - ConsumerRebalanceListenerCallbackNeededEvent that = (ConsumerRebalanceListenerCallbackNeededEvent) o; - - return methodName == that.methodName && partitions.equals(that.partitions); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + methodName.hashCode(); - result = 31 * result + partitions.hashCode(); - return result; - } - @Override protected String toStringBase() { return super.toStringBase() + ", methodName=" + methodName + ", partitions=" + partitions; } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java deleted file mode 100644 index 2945f22986..0000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorBackgroundEvent.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.clients.consumer.internals.events; - -import org.apache.kafka.common.KafkaException; - -public class ErrorBackgroundEvent extends BackgroundEvent { - - private final RuntimeException error; - - public ErrorBackgroundEvent(Throwable t) { - super(Type.ERROR); - this.error = t instanceof RuntimeException ? (RuntimeException) t : new KafkaException(t); - } - - public RuntimeException error() { - return error; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - ErrorBackgroundEvent that = (ErrorBackgroundEvent) o; - - return error.equals(that.error); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + error.hashCode(); - return result; - } - - @Override - public String toString() { - return "ErrorBackgroundEvent{" + - toStringBase() + - ", error=" + error + - '}'; - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorEvent.java new file mode 100644 index 0000000000..5e6d822382 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ErrorEvent.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +import org.apache.kafka.common.KafkaException; + +public class ErrorEvent extends BackgroundEvent { + + private final RuntimeException error; + + public ErrorEvent(Throwable t) { + super(Type.ERROR); + this.error = t instanceof RuntimeException ? (RuntimeException) t : new KafkaException(t); + } + + public RuntimeException error() { + return error; + } + + @Override + public String toStringBase() { + return super.toStringBase() + ", error=" + error; + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java deleted file mode 100644 index 34b2d97705..0000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsApplicationEvent.java +++ /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. - */ -package org.apache.kafka.clients.consumer.internals.events; - -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.common.TopicPartition; - -import java.util.Collections; -import java.util.Map; -import java.util.Set; - -public class FetchCommittedOffsetsApplicationEvent extends CompletableApplicationEvent> { - - /** - * Partitions to retrieve committed offsets for. - */ - private final Set partitions; - - /** - * Time until which the request will be retried if it fails with a retriable error. - */ - private final long timeoutMs; - - public FetchCommittedOffsetsApplicationEvent(final Set partitions, - final long timeoutMs) { - super(Type.FETCH_COMMITTED_OFFSETS); - this.partitions = Collections.unmodifiableSet(partitions); - this.timeoutMs = timeoutMs; - } - - public Set partitions() { - return partitions; - } - - public long timeout() { - return timeoutMs; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - FetchCommittedOffsetsApplicationEvent that = (FetchCommittedOffsetsApplicationEvent) o; - - return partitions.equals(that.partitions); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + partitions.hashCode(); - return result; - } - - @Override - public String toString() { - return getClass().getSimpleName() + "{" + - toStringBase() + - ", partitions=" + partitions + - ", timeout=" + timeoutMs + "ms" + - '}'; - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java similarity index 55% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java index 43dfee6ab1..980a8f1104 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java @@ -18,35 +18,30 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; + +import java.util.Collections; import java.util.Map; +import java.util.Set; -/** - * Event to commit offsets waiting for a response and retrying on expected retriable errors until - * the timer expires. - */ -public class SyncCommitApplicationEvent extends CommitApplicationEvent { +public class FetchCommittedOffsetsEvent extends CompletableApplicationEvent> { /** - * Time to wait for a response, retrying on retriable errors. + * Partitions to retrieve committed offsets for. */ - private final long retryTimeoutMs; + private final Set partitions; - public SyncCommitApplicationEvent(final Map offsets, - final long retryTimeoutMs) { - super(offsets, Type.COMMIT_SYNC); - this.retryTimeoutMs = retryTimeoutMs; + public FetchCommittedOffsetsEvent(final Set partitions, final Timer timer) { + super(Type.FETCH_COMMITTED_OFFSETS, timer); + this.partitions = Collections.unmodifiableSet(partitions); } - public Long retryTimeoutMs() { - return retryTimeoutMs; + public Set partitions() { + return partitions; } @Override - public String toString() { - return "SyncCommitApplicationEvent{" + - toStringBase() + - ", offsets=" + offsets() + - ", retryTimeout=" + retryTimeoutMs + "ms" + - '}'; + public String toStringBase() { + return super.toStringBase() + ", partitions=" + partitions; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java deleted file mode 100644 index 120e671724..0000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/GroupMetadataUpdateEvent.java +++ /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. - */ -package org.apache.kafka.clients.consumer.internals.events; - -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread; - -import java.util.Objects; - -/** - * This event is sent by the {@link ConsumerNetworkThread consumer's network thread} to the application thread - * so that when the user calls the {@link Consumer#groupMetadata()} API, the information is up-to-date. The - * information for the current state of the group member is managed on the consumer network thread and thus - * requires this interplay between threads. - */ -public class GroupMetadataUpdateEvent extends BackgroundEvent { - - final private int memberEpoch; - final private String memberId; - - public GroupMetadataUpdateEvent(final int memberEpoch, - final String memberId) { - super(Type.GROUP_METADATA_UPDATE); - this.memberEpoch = memberEpoch; - this.memberId = memberId; - } - - public int memberEpoch() { - return memberEpoch; - } - - public String memberId() { - return memberId; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - GroupMetadataUpdateEvent that = (GroupMetadataUpdateEvent) o; - return memberEpoch == that.memberEpoch && - Objects.equals(memberId, that.memberId); - } - - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), memberEpoch, memberId); - } - - @Override - public String toStringBase() { - return super.toStringBase() + - ", memberEpoch=" + memberEpoch + - ", memberId='" + memberId + '\''; - } - - @Override - public String toString() { - return "GroupMetadataUpdateEvent{" + - toStringBase() + - '}'; - } - -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java new file mode 100644 index 0000000000..e77b4dfb28 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +import org.apache.kafka.common.utils.Timer; + +public class LeaveOnCloseEvent extends CompletableApplicationEvent { + + public LeaveOnCloseEvent(final Timer timer) { + super(Type.LEAVE_ON_CLOSE, timer); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java similarity index 68% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java index 2466d06272..e218705846 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.HashMap; @@ -31,13 +32,13 @@ * {@link OffsetAndTimestamp} found (offset of the first message whose timestamp is greater than * or equals to the target timestamp) */ -public class ListOffsetsApplicationEvent extends CompletableApplicationEvent> { +public class ListOffsetsEvent extends CompletableApplicationEvent> { private final Map timestampsToSearch; private final boolean requireTimestamps; - public ListOffsetsApplicationEvent(Map timestampToSearch, boolean requireTimestamps) { - super(Type.LIST_OFFSETS); + public ListOffsetsEvent(final Map timestampToSearch, final boolean requireTimestamps, final Timer timer) { + super(Type.LIST_OFFSETS, timer); this.timestampsToSearch = Collections.unmodifiableMap(timestampToSearch); this.requireTimestamps = requireTimestamps; } @@ -64,31 +65,10 @@ public boolean requireTimestamps() { } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - ListOffsetsApplicationEvent that = (ListOffsetsApplicationEvent) o; - - if (requireTimestamps != that.requireTimestamps) return false; - return timestampsToSearch.equals(that.timestampsToSearch); - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + timestampsToSearch.hashCode(); - result = 31 * result + (requireTimestamps ? 1 : 0); - return result; - } - - @Override - public String toString() { - return getClass().getSimpleName() + " {" + - toStringBase() + - ", timestampsToSearch=" + timestampsToSearch + ", " + - "requireTimestamps=" + requireTimestamps + '}'; + public String toStringBase() { + return super.toStringBase() + + ", timestampsToSearch=" + timestampsToSearch + + ", requireTimestamps=" + requireTimestamps; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java index c06a3a717d..b06bd456f5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/NewTopicsMetadataUpdateRequestEvent.java @@ -21,11 +21,4 @@ public class NewTopicsMetadataUpdateRequestEvent extends ApplicationEvent { public NewTopicsMetadataUpdateRequestEvent() { super(Type.NEW_TOPICS_METADATA_UPDATE); } - - @Override - public String toString() { - return "NewTopicsMetadataUpdateRequestEvent{" + - toStringBase() + - '}'; - } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java deleted file mode 100644 index b958f0ec41..0000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollApplicationEvent.java +++ /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 org.apache.kafka.clients.consumer.internals.events; - -public class PollApplicationEvent extends ApplicationEvent { - - private final long pollTimeMs; - - public PollApplicationEvent(final long pollTimeMs) { - super(Type.POLL); - this.pollTimeMs = pollTimeMs; - } - - public long pollTimeMs() { - return pollTimeMs; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - - PollApplicationEvent that = (PollApplicationEvent) o; - - return pollTimeMs == that.pollTimeMs; - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + (int) (pollTimeMs ^ (pollTimeMs >>> 32)); - return result; - } - - @Override - public String toString() { - return "PollApplicationEvent{" + - toStringBase() + - ", pollTimeMs=" + pollTimeMs + - '}'; - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollEvent.java new file mode 100644 index 0000000000..96614c06e9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/PollEvent.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +public class PollEvent extends ApplicationEvent { + + private final long pollTimeMs; + + public PollEvent(final long pollTimeMs) { + super(Type.POLL); + this.pollTimeMs = pollTimeMs; + } + + public long pollTimeMs() { + return pollTimeMs; + } + + @Override + public String toStringBase() { + return super.toStringBase() + ", pollTimeMs=" + pollTimeMs; + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java similarity index 83% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java index 5d9b07f9de..65893b62ec 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java @@ -17,14 +17,16 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + /** * Event for resetting offsets for all assigned partitions that require it. This is an * asynchronous event that generates ListOffsets requests, and completes by updating in-memory * positions when responses are received. */ -public class ResetPositionsApplicationEvent extends CompletableApplicationEvent { +public class ResetPositionsEvent extends CompletableApplicationEvent { - public ResetPositionsApplicationEvent() { - super(Type.RESET_POSITIONS); + public ResetPositionsEvent(final Timer timer) { + super(Type.RESET_POSITIONS, timer); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeEvent.java similarity index 90% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeEvent.java index 73fd15fb14..ad5fd34c06 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SubscriptionChangeEvent.java @@ -22,9 +22,9 @@ * calls the subscribe API. This will make the consumer join a consumer group if not part of it * yet, or just send the updated subscription to the broker if it's already a member of the group. */ -public class SubscriptionChangeApplicationEvent extends ApplicationEvent { +public class SubscriptionChangeEvent extends ApplicationEvent { - public SubscriptionChangeApplicationEvent() { + public SubscriptionChangeEvent() { super(Type.SUBSCRIPTION_CHANGE); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java new file mode 100644 index 0000000000..87945616ea --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Timer; + +import java.util.Map; + +/** + * Event to commit offsets waiting for a response and retrying on expected retriable errors until + * the timer expires. + */ +public class SyncCommitEvent extends CommitEvent { + + public SyncCommitEvent(final Map offsets, final Timer timer) { + super(Type.COMMIT_SYNC, offsets, timer); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java deleted file mode 100644 index dd6f842cc2..0000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataApplicationEvent.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.clients.consumer.internals.events; - -import org.apache.kafka.common.PartitionInfo; - -import java.util.List; -import java.util.Map; -import java.util.Objects; - -public class TopicMetadataApplicationEvent extends CompletableApplicationEvent>> { - private final String topic; - private final boolean allTopics; - private final long timeoutMs; - - public TopicMetadataApplicationEvent(final long timeoutMs) { - super(Type.TOPIC_METADATA); - this.topic = null; - this.allTopics = true; - this.timeoutMs = timeoutMs; - } - - public TopicMetadataApplicationEvent(final String topic, final long timeoutMs) { - super(Type.TOPIC_METADATA); - this.topic = topic; - this.allTopics = false; - this.timeoutMs = timeoutMs; - } - - public String topic() { - return topic; - } - - public boolean isAllTopics() { - return allTopics; - } - - public long getTimeoutMs() { - return timeoutMs; - } - @Override - public String toString() { - return getClass().getSimpleName() + " {" + toStringBase() + - ", topic=" + topic + - ", allTopics=" + allTopics + - ", timeoutMs=" + timeoutMs + "}"; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (!(o instanceof TopicMetadataApplicationEvent)) return false; - if (!super.equals(o)) return false; - - TopicMetadataApplicationEvent that = (TopicMetadataApplicationEvent) o; - - return topic.equals(that.topic) && (allTopics == that.allTopics) && (timeoutMs == that.timeoutMs); - } - - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), topic, allTopics, timeoutMs); - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java new file mode 100644 index 0000000000..33e1270ce6 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +import org.apache.kafka.common.utils.Timer; + +import java.util.Objects; + +public class TopicMetadataEvent extends AbstractTopicMetadataEvent { + + private final String topic; + + public TopicMetadataEvent(final String topic, final Timer timer) { + super(Type.TOPIC_METADATA, timer); + this.topic = Objects.requireNonNull(topic); + } + + public String topic() { + return topic; + } + + @Override + public String toStringBase() { + return super.toStringBase() + ", topic=" + topic; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java similarity index 86% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java index a1ccb896fd..0b98837001 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java @@ -17,6 +17,8 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + /** * Application event triggered when a user calls the unsubscribe API. This will make the consumer * release all its assignments and send a heartbeat request to leave the consumer group. @@ -24,9 +26,10 @@ * complete and the heartbeat to leave the group is sent out (minimal effort to send the * leave group heartbeat, without waiting for any response or considering timeouts). */ -public class UnsubscribeApplicationEvent extends CompletableApplicationEvent { - public UnsubscribeApplicationEvent() { - super(Type.UNSUBSCRIBE); +public class UnsubscribeEvent extends CompletableApplicationEvent { + + public UnsubscribeEvent(final Timer timer) { + super(Type.UNSUBSCRIBE, timer); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java similarity index 83% rename from clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java rename to clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java index 3b093e0b68..21e7f3cf6e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java @@ -17,14 +17,16 @@ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Timer; + /** * Event for validating offsets for all assigned partitions for which a leader change has been * detected. This is an asynchronous event that generates OffsetForLeaderEpoch requests, and * completes by validating in-memory positions against the offsets received in the responses. */ -public class ValidatePositionsApplicationEvent extends CompletableApplicationEvent { +public class ValidatePositionsEvent extends CompletableApplicationEvent { - public ValidatePositionsApplicationEvent() { - super(Type.VALIDATE_POSITIONS); + public ValidatePositionsEvent(final Timer timer) { + super(Type.VALIDATE_POSITIONS, timer); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/metrics/RebalanceMetricsManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/metrics/RebalanceMetricsManager.java new file mode 100644 index 0000000000..a255487f37 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/metrics/RebalanceMetricsManager.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.metrics; + +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Measurable; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.CumulativeCount; +import org.apache.kafka.common.metrics.stats.CumulativeSum; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.metrics.stats.WindowedCount; + +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX; +import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.COORDINATOR_METRICS_SUFFIX; + +public class RebalanceMetricsManager { + private final Sensor successfulRebalanceSensor; + private final Sensor failedRebalanceSensor; + private final String metricGroupName; + + public final MetricName rebalanceLatencyAvg; + public final MetricName rebalanceLatencyMax; + public final MetricName rebalanceLatencyTotal; + public final MetricName rebalanceTotal; + public final MetricName rebalanceRatePerHour; + public final MetricName lastRebalanceSecondsAgo; + public final MetricName failedRebalanceTotal; + public final MetricName failedRebalanceRate; + private long lastRebalanceEndMs = -1L; + private long lastRebalanceStartMs = -1L; + + public RebalanceMetricsManager(Metrics metrics) { + metricGroupName = CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX; + + rebalanceLatencyAvg = createMetric(metrics, "rebalance-latency-avg", + "The average time taken for a group to complete a rebalance"); + rebalanceLatencyMax = createMetric(metrics, "rebalance-latency-max", + "The max time taken for a group to complete a rebalance"); + rebalanceLatencyTotal = createMetric(metrics, "rebalance-latency-total", + "The total number of milliseconds spent in rebalances"); + rebalanceTotal = createMetric(metrics, "rebalance-total", + "The total number of rebalance events"); + rebalanceRatePerHour = createMetric(metrics, "rebalance-rate-per-hour", + "The number of rebalance events per hour"); + failedRebalanceTotal = createMetric(metrics, "failed-rebalance-total", + "The total number of failed rebalance events"); + failedRebalanceRate = createMetric(metrics, "failed-rebalance-rate-per-hour", + "The number of failed rebalance events per hour"); + + successfulRebalanceSensor = metrics.sensor("rebalance-latency"); + successfulRebalanceSensor.add(rebalanceLatencyAvg, new Avg()); + successfulRebalanceSensor.add(rebalanceLatencyMax, new Max()); + successfulRebalanceSensor.add(rebalanceLatencyTotal, new CumulativeSum()); + successfulRebalanceSensor.add(rebalanceTotal, new CumulativeCount()); + successfulRebalanceSensor.add(rebalanceRatePerHour, new Rate(TimeUnit.HOURS, new WindowedCount())); + + failedRebalanceSensor = metrics.sensor("failed-rebalance"); + failedRebalanceSensor.add(failedRebalanceTotal, new CumulativeSum()); + failedRebalanceSensor.add(failedRebalanceRate, new Rate(TimeUnit.HOURS, new WindowedCount())); + + Measurable lastRebalance = (config, now) -> { + if (lastRebalanceEndMs == -1L) + return -1d; + else + return TimeUnit.SECONDS.convert(now - lastRebalanceEndMs, TimeUnit.MILLISECONDS); + }; + lastRebalanceSecondsAgo = createMetric(metrics, + "last-rebalance-seconds-ago", + "The number of seconds since the last rebalance event"); + metrics.addMetric(lastRebalanceSecondsAgo, lastRebalance); + } + + private MetricName createMetric(Metrics metrics, String name, String description) { + return metrics.metricName(name, metricGroupName, description); + } + + public void recordRebalanceStarted(long nowMs) { + lastRebalanceStartMs = nowMs; + } + + public void recordRebalanceEnded(long nowMs) { + lastRebalanceEndMs = nowMs; + successfulRebalanceSensor.record(nowMs - lastRebalanceStartMs); + } + + public void maybeRecordRebalanceFailed() { + if (lastRebalanceStartMs <= lastRebalanceEndMs) + return; + failedRebalanceSensor.record(); + } + + public boolean rebalanceStarted() { + return lastRebalanceStartMs > lastRebalanceEndMs; + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index d74b5e4c1d..9471b48aa4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -294,8 +294,10 @@ public class ProducerConfig extends AbstractConfig { "

    " + "
  • If not set, the default partitioning logic is used. " + "This strategy send records to a partition until at least " + BATCH_SIZE_CONFIG + " bytes is produced to the partition. It works with the strategy:" + - "

    1) If no partition is specified but a key is present, choose a partition based on a hash of the key." + - "

    2) If no partition or key is present, choose the sticky partition that changes when at least " + BATCH_SIZE_CONFIG + " bytes are produced to the partition." + + "

      " + + "
    1. If no partition is specified but a key is present, choose a partition based on a hash of the key.
    2. " + + "
    3. If no partition or key is present, choose the sticky partition that changes when at least " + BATCH_SIZE_CONFIG + " bytes are produced to the partition.
    4. " + + "
    " + "
  • " + "
  • org.apache.kafka.clients.producer.RoundRobinPartitioner: A partitioning strategy where " + "each record in a series of consecutive records is sent to a different partition, regardless of whether the 'key' is provided or not, " + 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 2cdc3b876d..99bc1d68b0 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 @@ -706,9 +706,8 @@ private void completeBatch(ProducerBatch batch, ProduceResponse.PartitionRespons "topic-partition may not exist or the user may not have Describe access to it", batch.topicPartition); } else { - log.warn("Received invalid metadata error in produce request on partition {} due to {}. Going " + - "to request metadata update now", batch.topicPartition, - error.exception(response.errorMessage).toString()); + log.warn("Received invalid metadata error in produce request on partition {} due to {} Going " + + "to request metadata update now", batch.topicPartition, error.exception(response.errorMessage).toString()); } if (error.exception() instanceof NotLeaderOrFollowerException || error.exception() instanceof FencedLeaderEpochException) { log.debug("For {}, received error {}, with leaderIdAndEpoch {}", batch.topicPartition, error, response.currentLeader); 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 14a519a7a0..4ff54b1759 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 @@ -1330,6 +1330,8 @@ public void handleResponse(AbstractResponse response) { // We could still receive INVALID_PRODUCER_EPOCH from old versioned transaction coordinator, // just treat it the same as PRODUCE_FENCED. fatalError(Errors.PRODUCER_FENCED.exception()); + } else if (error == Errors.ABORTABLE_TRANSACTION) { + abortableError(error.exception()); } else { fatalError(new KafkaException("Unexpected error in InitProducerIdResponse; " + error.message())); } @@ -1386,10 +1388,8 @@ public void handleResponse(AbstractResponse response) { // just treat it the same as PRODUCE_FENCED. fatalError(Errors.PRODUCER_FENCED.exception()); return; - } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) { - fatalError(error.exception()); - return; - } else if (error == Errors.INVALID_TXN_STATE) { + } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED || + error == Errors.INVALID_TXN_STATE) { fatalError(error.exception()); return; } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) { @@ -1401,6 +1401,9 @@ public void handleResponse(AbstractResponse response) { } else if (error == Errors.UNKNOWN_PRODUCER_ID || error == Errors.INVALID_PRODUCER_ID_MAPPING) { abortableErrorIfPossible(error.exception()); return; + } else if (error == Errors.ABORTABLE_TRANSACTION) { + abortableError(error.exception()); + return; } else { log.error("Could not add partition {} due to unexpected error {}", topicPartition, error); hasPartitionErrors = true; @@ -1504,6 +1507,8 @@ public void handleResponse(AbstractResponse response) { fatalError(error.exception()); } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { abortableError(GroupAuthorizationException.forGroupId(key)); + } else if (error == Errors.ABORTABLE_TRANSACTION) { + abortableError(error.exception()); } else { fatalError(new KafkaException(String.format("Could not find a coordinator with type %s with key %s due to " + "unexpected error: %s", coordinatorType, key, @@ -1552,12 +1557,13 @@ public void handleResponse(AbstractResponse response) { // We could still receive INVALID_PRODUCER_EPOCH from old versioned transaction coordinator, // just treat it the same as PRODUCE_FENCED. fatalError(Errors.PRODUCER_FENCED.exception()); - } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) { - fatalError(error.exception()); - } else if (error == Errors.INVALID_TXN_STATE) { + } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED || + error == Errors.INVALID_TXN_STATE) { fatalError(error.exception()); } else if (error == Errors.UNKNOWN_PRODUCER_ID || error == Errors.INVALID_PRODUCER_ID_MAPPING) { abortableErrorIfPossible(error.exception()); + } else if (error == Errors.ABORTABLE_TRANSACTION) { + abortableError(error.exception()); } else { fatalError(new KafkaException("Unhandled error in EndTxnResponse: " + error.message())); } @@ -1611,12 +1617,13 @@ public void handleResponse(AbstractResponse response) { // We could still receive INVALID_PRODUCER_EPOCH from old versioned transaction coordinator, // just treat it the same as PRODUCE_FENCED. fatalError(Errors.PRODUCER_FENCED.exception()); - } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) { - fatalError(error.exception()); - } else if (error == Errors.INVALID_TXN_STATE) { + } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED || + error == Errors.INVALID_TXN_STATE) { fatalError(error.exception()); } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { abortableError(GroupAuthorizationException.forGroupId(builder.data.groupId())); + } else if (error == Errors.ABORTABLE_TRANSACTION) { + abortableError(error.exception()); } else { fatalError(new KafkaException("Unexpected error in AddOffsetsToTxnResponse: " + error.message())); } @@ -1679,7 +1686,8 @@ public void handleResponse(AbstractResponse response) { } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { abortableError(GroupAuthorizationException.forGroupId(builder.data.groupId())); break; - } else if (error == Errors.FENCED_INSTANCE_ID) { + } else if (error == Errors.FENCED_INSTANCE_ID || + error == Errors.ABORTABLE_TRANSACTION) { abortableError(error.exception()); break; } else if (error == Errors.UNKNOWN_MEMBER_ID diff --git a/clients/src/main/java/org/apache/kafka/common/errors/AbortableTransactionException.java b/clients/src/main/java/org/apache/kafka/common/errors/AbortableTransactionException.java new file mode 100644 index 0000000000..cfea649e75 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/AbortableTransactionException.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +public class AbortableTransactionException extends ApiException { + public AbortableTransactionException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java b/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java index 6d395fa3fb..c8e53ffc6c 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java @@ -26,9 +26,17 @@ public final class KafkaMetric implements Metric { private final Object lock; private final Time time; private final MetricValueProvider metricValueProvider; - private MetricConfig config; + private volatile MetricConfig config; // public for testing + /** + * Create a metric to monitor an object that implements MetricValueProvider. + * @param lock The lock used to prevent race condition + * @param metricName The name of the metric + * @param valueProvider The metric value provider associated with this metric + * @param config The configuration of the metric + * @param time The time instance to use with the metrics + */ public KafkaMetric(Object lock, MetricName metricName, MetricValueProvider valueProvider, MetricConfig config, Time time) { this.metricName = metricName; @@ -40,15 +48,29 @@ public KafkaMetric(Object lock, MetricName metricName, MetricValueProvider va this.time = time; } + /** + * Get the configuration of this metric. + * This is supposed to be used by server only. + * @return Return the config of this metric + */ public MetricConfig config() { return this.config; } + /** + * Get the metric name + * @return Return the name of this metric + */ @Override public MetricName metricName() { return this.metricName; } + /** + * Take the metric and return the value, which could be a {@link Measurable} or a {@link Gauge} + * @return Return the metric value + * @throws IllegalStateException if the underlying metric is not a {@link Measurable} or a {@link Gauge}. + */ @Override public Object metricValue() { long now = time.milliseconds(); @@ -62,6 +84,11 @@ else if (this.metricValueProvider instanceof Gauge) } } + /** + * Get the underlying metric provider, which should be a {@link Measurable} + * @return Return the metric provider + * @throws IllegalStateException if the underlying metric is not a {@link Measurable}. + */ public Measurable measurable() { if (this.metricValueProvider instanceof Measurable) return (Measurable) metricValueProvider; @@ -69,6 +96,11 @@ public Measurable measurable() { throw new IllegalStateException("Not a measurable: " + this.metricValueProvider.getClass()); } + /** + * Take the metric and return the value, where the underlying metric provider should be a {@link Measurable} + * @param timeMs The time that this metric is taken + * @return Return the metric value if it's measurable, otherwise 0 + */ double measurableValue(long timeMs) { synchronized (this.lock) { if (this.metricValueProvider instanceof Measurable) @@ -78,6 +110,11 @@ public Measurable measurable() { } } + /** + * Set the metric config. + * This is supposed to be used by server only. + * @param config configuration for this metrics + */ public void config(MetricConfig config) { synchronized (lock) { this.config = config; 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 6447cdb5c7..b52285dac6 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 @@ -277,40 +277,30 @@ public static String toHtmlTable(String domain, Iterable all } } } - StringBuilder b = new StringBuilder(); - b.append("\n"); - + b.append("
    \n\n"); + b.append("\n"); + b.append("\n"); + b.append("\n"); + b.append("\n"); + b.append("\n"); for (Entry> e : beansAndAttributes.entrySet()) { - b.append("\n"); - b.append(""); - b.append("\n"); - - b.append("\n"); - b.append("\n"); - b.append("\n"); - b.append("\n"); - b.append("\n"); - for (Entry e2 : e.getValue().entrySet()) { b.append("\n"); - b.append(""); b.append(""); + b.append("\n"); b.append(""); + b.append("\n"); + b.append("\n"); b.append("\n"); } - } b.append("
    Metric/Attribute nameDescriptionMbean name
    "); - b.append(e.getKey()); - b.append("
    Attribute nameDescription
    "); b.append(e2.getKey()); - b.append(""); b.append(e2.getValue()); - b.append(""); + b.append(e.getKey()); + b.append("
    "); - return b.toString(); - } public MetricConfig config() { 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 904c5216a4..da80e363a9 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 @@ -498,13 +498,14 @@ private SSLEngineResult handshakeWrap(boolean doWrite) throws IOException { } /** - * Perform handshake unwrap + * Perform handshake unwrap. + * Visible for testing. * @param doRead boolean If true, read more from the socket channel * @param ignoreHandshakeStatus If true, continue to unwrap if data available regardless of handshake status * @return SSLEngineResult * @throws IOException */ - private SSLEngineResult handshakeUnwrap(boolean doRead, boolean ignoreHandshakeStatus) throws IOException { + SSLEngineResult handshakeUnwrap(boolean doRead, boolean ignoreHandshakeStatus) throws IOException { log.trace("SSLHandshake handshakeUnwrap {}", channelId); SSLEngineResult result; int read = 0; @@ -526,7 +527,7 @@ private SSLEngineResult handshakeUnwrap(boolean doRead, boolean ignoreHandshakeS handshakeStatus == HandshakeStatus.NEED_UNWRAP) || (ignoreHandshakeStatus && netReadBuffer.position() != position); log.trace("SSLHandshake handshakeUnwrap: handshakeStatus {} status {}", handshakeStatus, result.getStatus()); - } while (netReadBuffer.position() != 0 && cont); + } while (cont); // Throw EOF exception for failed read after processing already received data // so that handshake failures are reported correctly 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 acc99cdbec..e0ece7b2a3 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 @@ -143,6 +143,9 @@ import org.apache.kafka.common.errors.UnsupportedForMessageFormatException; import org.apache.kafka.common.errors.UnsupportedSaslMechanismException; import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.errors.AbortableTransactionException; + +// AutoMQ inject start import org.apache.kafka.common.errors.s3.CompactedObjectsNotFoundException; import org.apache.kafka.common.errors.s3.KeyExistException; import org.apache.kafka.common.errors.s3.NodeEpochExpiredException; @@ -157,6 +160,8 @@ import org.apache.kafka.common.errors.s3.StreamNotClosedException; import org.apache.kafka.common.errors.s3.StreamNotExistException; import org.apache.kafka.common.errors.s3.StreamNotOpenedException; +// AutoMQ inject end + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -376,12 +381,12 @@ public enum Errors { DUPLICATE_RESOURCE(92, "A request illegally referred to the same resource twice.", DuplicateResourceException::new), UNACCEPTABLE_CREDENTIAL(93, "Requested credential would not meet criteria for acceptability.", UnacceptableCredentialException::new), INCONSISTENT_VOTER_SET(94, "Indicates that the either the sender or recipient of a " + - "voter-only request is not one of the expected voters", InconsistentVoterSetException::new), + "voter-only request is not one of the expected voters.", InconsistentVoterSetException::new), INVALID_UPDATE_VERSION(95, "The given update version was invalid.", InvalidUpdateVersionException::new), FEATURE_UPDATE_FAILED(96, "Unable to update finalized features due to an unexpected server error.", FeatureUpdateFailedException::new), PRINCIPAL_DESERIALIZATION_FAILURE(97, "Request principal deserialization failed during forwarding. " + "This indicates an internal error on the broker cluster security setup.", PrincipalDeserializationException::new), - SNAPSHOT_NOT_FOUND(98, "Requested snapshot was not found", SnapshotNotFoundException::new), + SNAPSHOT_NOT_FOUND(98, "Requested snapshot was not found.", SnapshotNotFoundException::new), POSITION_OUT_OF_RANGE( 99, "Requested position is not greater than or equal to zero, and less than the size of the snapshot.", @@ -389,10 +394,10 @@ public enum Errors { UNKNOWN_TOPIC_ID(100, "This server does not host this topic ID.", UnknownTopicIdException::new), DUPLICATE_BROKER_REGISTRATION(101, "This broker ID is already in use.", DuplicateBrokerRegistrationException::new), BROKER_ID_NOT_REGISTERED(102, "The given broker ID was not registered.", BrokerIdNotRegisteredException::new), - INCONSISTENT_TOPIC_ID(103, "The log's topic ID did not match the topic ID in the request", InconsistentTopicIdException::new), - INCONSISTENT_CLUSTER_ID(104, "The clusterId in the request does not match that found on the server", InconsistentClusterIdException::new), - TRANSACTIONAL_ID_NOT_FOUND(105, "The transactionalId could not be found", TransactionalIdNotFoundException::new), - FETCH_SESSION_TOPIC_ID_ERROR(106, "The fetch session encountered inconsistent topic ID usage", FetchSessionTopicIdException::new), + INCONSISTENT_TOPIC_ID(103, "The log's topic ID did not match the topic ID in the request.", InconsistentTopicIdException::new), + INCONSISTENT_CLUSTER_ID(104, "The clusterId in the request does not match that found on the server.", InconsistentClusterIdException::new), + TRANSACTIONAL_ID_NOT_FOUND(105, "The transactionalId could not be found.", TransactionalIdNotFoundException::new), + FETCH_SESSION_TOPIC_ID_ERROR(106, "The fetch session encountered inconsistent topic ID usage.", FetchSessionTopicIdException::new), INELIGIBLE_REPLICA(107, "The new ISR contains at least one ineligible replica.", IneligibleReplicaException::new), NEW_LEADER_ELECTED(108, "The AlterPartition request successfully updated the partition state but the leader has changed.", NewLeaderElectedException::new), OFFSET_MOVED_TO_TIERED_STORAGE(109, "The requested offset is moved to tiered storage.", OffsetMovedToTieredStorageException::new), @@ -406,6 +411,7 @@ public enum Errors { UNKNOWN_SUBSCRIPTION_ID(117, "Client sent a push telemetry request with an invalid or outdated subscription ID.", UnknownSubscriptionIdException::new), TELEMETRY_TOO_LARGE(118, "Client sent a push telemetry request larger than the maximum size the broker will accept.", TelemetryTooLargeException::new), INVALID_REGISTRATION(119, "The controller has considered the broker registration to be invalid.", InvalidRegistrationException::new), + ABORTABLE_TRANSACTION(120, "The server encountered an error with the transaction. The client can abort the transaction to continue using this transactional ID.", AbortableTransactionException::new), // AutoMQ for Kafka inject start STREAM_EXIST(501, "The stream already exists.", StreamExistException::new), @@ -423,7 +429,7 @@ public enum Errors { KEY_NOT_EXIST(513, "The key does not exist.", ObjectNotExistException::new), NODE_FENCED(514, "The node is fenced.", NodeFencedException::new), STREAM_INNER_ERROR(599, "The stream inner error.", StreamInnerErrorException::new); - // AutoMQ for Kafka inject end + // AutoMQ inject end private static final Logger log = LoggerFactory.getLogger(Errors.class); 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 3aa233c34e..3ba60b09b3 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 @@ -209,7 +209,7 @@ private static FilterResult filterTo(TopicPartition partition, Iterable this.maxTimestamp) { this.maxTimestamp = maxTimestamp; - this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp; + this.offsetOfMaxTimestamp = offsetOfMaxTimestamp; } 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); + private void validateBatchMetadata(long maxTimestamp, long offsetOfMaxTimestamp, long maxOffset) { + if (maxTimestamp != RecordBatch.NO_TIMESTAMP && offsetOfMaxTimestamp < 0) + throw new IllegalArgumentException("offset undefined for maximum timestamp " + maxTimestamp); if (maxOffset < 0) throw new IllegalArgumentException("maxOffset undefined"); } @@ -458,8 +458,8 @@ public long maxTimestamp() { return maxTimestamp; } - public long shallowOffsetOfMaxTimestamp() { - return shallowOffsetOfMaxTimestamp; + public long offsetOfMaxTimestamp() { + return offsetOfMaxTimestamp; } } diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index 143a515bcd..6b53ee4159 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -242,34 +242,23 @@ public MemoryRecords build() { /** * Get the max timestamp and its offset. The details of the offset returned are a bit subtle. + * Note: The semantic for the offset of max timestamp is the first offset with the max timestamp if there are multi-records having same timestamp. * - * If the log append time is used, the offset will be the last offset unless no compression is used and - * the message format version is 0 or 1, in which case, it will be the first offset. + * If the log append time is used, the offset will be the first offset of the record. * - * If create time is used, the offset will be the last offset unless no compression is used and the message - * format version is 0 or 1, in which case, it will be the offset of the record with the max timestamp. + * If create time is used, the offset will always be the offset of the record with the max timestamp. + * + * If it's NO_TIMESTAMP (i.e. MAGIC_VALUE_V0), we'll return offset -1 since no timestamp info in records. * * @return The max timestamp and its offset */ public RecordsInfo info() { if (timestampType == TimestampType.LOG_APPEND_TIME) { - long shallowOffsetOfMaxTimestamp; - // Use the last offset when dealing with record batches - if (compressionType != CompressionType.NONE || magic >= RecordBatch.MAGIC_VALUE_V2) - shallowOffsetOfMaxTimestamp = lastOffset; - else - shallowOffsetOfMaxTimestamp = baseOffset; - return new RecordsInfo(logAppendTime, shallowOffsetOfMaxTimestamp); - } else if (maxTimestamp == RecordBatch.NO_TIMESTAMP) { - return new RecordsInfo(RecordBatch.NO_TIMESTAMP, lastOffset); + return new RecordsInfo(logAppendTime, baseOffset); } else { - long shallowOffsetOfMaxTimestamp; - // Use the last offset when dealing with record batches - if (compressionType != CompressionType.NONE || magic >= RecordBatch.MAGIC_VALUE_V2) - shallowOffsetOfMaxTimestamp = lastOffset; - else - shallowOffsetOfMaxTimestamp = offsetOfMaxTimestamp; - return new RecordsInfo(maxTimestamp, shallowOffsetOfMaxTimestamp); + // For create time, we always use offsetOfMaxTimestamp for the correct time -> offset mapping + // If it's MAGIC_VALUE_V0, the value will be the default value: [-1, -1] + return new RecordsInfo(maxTimestamp, offsetOfMaxTimestamp); } } @@ -862,12 +851,12 @@ private long nextSequentialOffset() { public static class RecordsInfo { public final long maxTimestamp; - public final long shallowOffsetOfMaxTimestamp; + public final long offsetOfMaxTimestamp; public RecordsInfo(long maxTimestamp, - long shallowOffsetOfMaxTimestamp) { + long offsetOfMaxTimestamp) { this.maxTimestamp = maxTimestamp; - this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp; + this.offsetOfMaxTimestamp = offsetOfMaxTimestamp; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java index efdc7da2af..fc996453d6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java @@ -47,6 +47,8 @@ public class ListOffsetsRequest extends AbstractRequest { */ public static final long EARLIEST_LOCAL_TIMESTAMP = -4L; + public static final long LATEST_TIERED_TIMESTAMP = -5L; + public static final int CONSUMER_REPLICA_ID = -1; public static final int DEBUGGING_REPLICA_ID = -2; 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 index 73bfcd15c1..3b340131cf 100644 --- 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 @@ -34,7 +34,7 @@ public class OAuthBearerClientInitialResponse { 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 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)); public static final String AUTH_KEY = "auth"; diff --git a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java index ae92c539da..483179d4c4 100644 --- a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java +++ b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java @@ -265,8 +265,8 @@ class DefaultClientTelemetrySender implements ClientTelemetrySender { These are the lower and upper bounds of the jitter that we apply to the initial push telemetry API call. This helps to avoid a flood of requests all coming at the same time. */ - private final static double INITIAL_PUSH_JITTER_LOWER = 0.5; - private final static double INITIAL_PUSH_JITTER_UPPER = 1.5; + private static final double INITIAL_PUSH_JITTER_LOWER = 0.5; + private static final double INITIAL_PUSH_JITTER_UPPER = 1.5; private final ReadWriteLock lock = new ReentrantReadWriteLock(); private final Condition subscriptionLoaded = lock.writeLock().newCondition(); @@ -325,6 +325,7 @@ public long timeToNextUpdate(long requestTimeoutMs) { final long timeMs; final String apiName; final String msg; + final boolean isTraceEnabled = log.isTraceEnabled(); switch (localState) { case SUBSCRIPTION_IN_PROGRESS: @@ -336,15 +337,15 @@ public long timeToNextUpdate(long requestTimeoutMs) { */ apiName = (localState == ClientTelemetryState.SUBSCRIPTION_IN_PROGRESS) ? ApiKeys.GET_TELEMETRY_SUBSCRIPTIONS.name : ApiKeys.PUSH_TELEMETRY.name; timeMs = requestTimeoutMs; - msg = String.format("the remaining wait time for the %s network API request, as specified by %s", apiName, CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG); + msg = isTraceEnabled ? "" : String.format("the remaining wait time for the %s network API request, as specified by %s", apiName, CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG); break; case TERMINATING_PUSH_IN_PROGRESS: timeMs = Long.MAX_VALUE; - msg = String.format("the terminating push is in progress, disabling telemetry for further requests"); + msg = isTraceEnabled ? "" : "the terminating push is in progress, disabling telemetry for further requests"; break; case TERMINATING_PUSH_NEEDED: timeMs = 0; - msg = String.format("the client should try to submit the final %s network API request ASAP before closing", ApiKeys.PUSH_TELEMETRY.name); + msg = isTraceEnabled ? "" : String.format("the client should try to submit the final %s network API request ASAP before closing", ApiKeys.PUSH_TELEMETRY.name); break; case SUBSCRIPTION_NEEDED: case PUSH_NEEDED: @@ -352,17 +353,19 @@ public long timeToNextUpdate(long requestTimeoutMs) { long timeRemainingBeforeRequest = localLastRequestMs + localIntervalMs - nowMs; if (timeRemainingBeforeRequest <= 0) { timeMs = 0; - msg = String.format("the wait time before submitting the next %s network API request has elapsed", apiName); + msg = isTraceEnabled ? "" : String.format("the wait time before submitting the next %s network API request has elapsed", apiName); } else { timeMs = timeRemainingBeforeRequest; - msg = String.format("the client will wait before submitting the next %s network API request", apiName); + msg = isTraceEnabled ? "" : String.format("the client will wait before submitting the next %s network API request", apiName); } break; default: throw new IllegalStateException("Unknown telemetry state: " + localState); } - log.trace("For telemetry state {}, returning the value {} ms; {}", localState, timeMs, msg); + if (isTraceEnabled) { + log.trace("For telemetry state {}, returning the value {} ms; {}", localState, timeMs, msg); + } return timeMs; } diff --git a/clients/src/main/resources/common/message/AddOffsetsToTxnRequest.json b/clients/src/main/resources/common/message/AddOffsetsToTxnRequest.json index ade3fc72c9..9d7b63c313 100644 --- a/clients/src/main/resources/common/message/AddOffsetsToTxnRequest.json +++ b/clients/src/main/resources/common/message/AddOffsetsToTxnRequest.json @@ -23,7 +23,9 @@ // Version 2 adds the support for new error code PRODUCER_FENCED. // // Version 3 enables flexible versions. - "validVersions": "0-3", + // + // Version 4 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId", diff --git a/clients/src/main/resources/common/message/AddOffsetsToTxnResponse.json b/clients/src/main/resources/common/message/AddOffsetsToTxnResponse.json index 71fa655227..6b3b1c481d 100644 --- a/clients/src/main/resources/common/message/AddOffsetsToTxnResponse.json +++ b/clients/src/main/resources/common/message/AddOffsetsToTxnResponse.json @@ -22,7 +22,9 @@ // Version 2 adds the support for new error code PRODUCER_FENCED. // // Version 3 enables flexible versions. - "validVersions": "0-3", + // + // Version 4 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", diff --git a/clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json b/clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json index 1b89c54d86..139d1436a6 100644 --- a/clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json +++ b/clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json @@ -25,9 +25,11 @@ // Version 3 enables flexible versions. // // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions. + // + // Version 5 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). // Versions 3 and below will be exclusively used by clients and versions 4 and above will be used by brokers. "latestVersionUnstable": false, - "validVersions": "0-4", + "validVersions": "0-5", "flexibleVersions": "3+", "fields": [ { "name": "Transactions", "type": "[]AddPartitionsToTxnTransaction", "versions": "4+", diff --git a/clients/src/main/resources/common/message/AddPartitionsToTxnResponse.json b/clients/src/main/resources/common/message/AddPartitionsToTxnResponse.json index 326b4acdb4..a2af388dba 100644 --- a/clients/src/main/resources/common/message/AddPartitionsToTxnResponse.json +++ b/clients/src/main/resources/common/message/AddPartitionsToTxnResponse.json @@ -24,7 +24,9 @@ // Version 3 enables flexible versions. // // Version 4 adds support to batch multiple transactions and a top level error code. - "validVersions": "0-4", + // + // Version 5 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). + "validVersions": "0-5", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", diff --git a/clients/src/main/resources/common/message/EndTxnRequest.json b/clients/src/main/resources/common/message/EndTxnRequest.json index f16ef76246..5bf6c57734 100644 --- a/clients/src/main/resources/common/message/EndTxnRequest.json +++ b/clients/src/main/resources/common/message/EndTxnRequest.json @@ -23,7 +23,9 @@ // Version 2 adds the support for new error code PRODUCER_FENCED. // // Version 3 enables flexible versions. - "validVersions": "0-3", + // + // Version 4 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId", diff --git a/clients/src/main/resources/common/message/EndTxnResponse.json b/clients/src/main/resources/common/message/EndTxnResponse.json index 3071953185..53b0250f80 100644 --- a/clients/src/main/resources/common/message/EndTxnResponse.json +++ b/clients/src/main/resources/common/message/EndTxnResponse.json @@ -22,7 +22,9 @@ // Version 2 adds the support for new error code PRODUCER_FENCED. // // Version 3 enables flexible versions. - "validVersions": "0-3", + // + // Version 4 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", diff --git a/clients/src/main/resources/common/message/FindCoordinatorRequest.json b/clients/src/main/resources/common/message/FindCoordinatorRequest.json index 49ae7e92f0..e6786f5b10 100644 --- a/clients/src/main/resources/common/message/FindCoordinatorRequest.json +++ b/clients/src/main/resources/common/message/FindCoordinatorRequest.json @@ -25,7 +25,9 @@ // Version 3 is the first flexible version. // // Version 4 adds support for batching via CoordinatorKeys (KIP-699) - "validVersions": "0-4", + // + // Version 5 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). + "validVersions": "0-5", "deprecatedVersions": "0", "flexibleVersions": "3+", "fields": [ diff --git a/clients/src/main/resources/common/message/FindCoordinatorResponse.json b/clients/src/main/resources/common/message/FindCoordinatorResponse.json index 9309c0177d..a744a1928d 100644 --- a/clients/src/main/resources/common/message/FindCoordinatorResponse.json +++ b/clients/src/main/resources/common/message/FindCoordinatorResponse.json @@ -24,7 +24,9 @@ // Version 3 is the first flexible version. // // Version 4 adds support for batching via Coordinators (KIP-699) - "validVersions": "0-4", + // + // Version 5 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). + "validVersions": "0-5", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, diff --git a/clients/src/main/resources/common/message/InitProducerIdRequest.json b/clients/src/main/resources/common/message/InitProducerIdRequest.json index 4e75352db6..92e6a6b253 100644 --- a/clients/src/main/resources/common/message/InitProducerIdRequest.json +++ b/clients/src/main/resources/common/message/InitProducerIdRequest.json @@ -25,7 +25,9 @@ // Version 3 adds ProducerId and ProducerEpoch, allowing producers to try to resume after an INVALID_PRODUCER_EPOCH error // // Version 4 adds the support for new error code PRODUCER_FENCED. - "validVersions": "0-4", + // + // Verison 5 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). + "validVersions": "0-5", "flexibleVersions": "2+", "fields": [ { "name": "TransactionalId", "type": "string", "versions": "0+", "nullableVersions": "0+", "entityType": "transactionalId", diff --git a/clients/src/main/resources/common/message/InitProducerIdResponse.json b/clients/src/main/resources/common/message/InitProducerIdResponse.json index f56c2fe67e..c0f10b2e85 100644 --- a/clients/src/main/resources/common/message/InitProducerIdResponse.json +++ b/clients/src/main/resources/common/message/InitProducerIdResponse.json @@ -24,7 +24,9 @@ // Version 3 is the same as version 2. // // Version 4 adds the support for new error code PRODUCER_FENCED. - "validVersions": "0-4", + // + // Version 5 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). + "validVersions": "0-5", "flexibleVersions": "2+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, diff --git a/clients/src/main/resources/common/message/ProduceRequest.json b/clients/src/main/resources/common/message/ProduceRequest.json index 3451a8471d..d396d66070 100644 --- a/clients/src/main/resources/common/message/ProduceRequest.json +++ b/clients/src/main/resources/common/message/ProduceRequest.json @@ -35,7 +35,9 @@ // Version 9 enables flexible versions. // // Version 10 is the same as version 9 (KIP-951). - "validVersions": "0-10", + // + // Version 11 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). + "validVersions": "0-11", "deprecatedVersions": "0-6", "flexibleVersions": "9+", "fields": [ diff --git a/clients/src/main/resources/common/message/ProduceResponse.json b/clients/src/main/resources/common/message/ProduceResponse.json index d294fb8aa2..1c097d3bc3 100644 --- a/clients/src/main/resources/common/message/ProduceResponse.json +++ b/clients/src/main/resources/common/message/ProduceResponse.json @@ -34,7 +34,9 @@ // Version 9 enables flexible versions. // // Version 10 adds 'CurrentLeader' and 'NodeEndpoints' as tagged fields (KIP-951) - "validVersions": "0-10", + // + // Version 11 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). + "validVersions": "0-11", "flexibleVersions": "9+", "fields": [ { "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+", diff --git a/clients/src/main/resources/common/message/TxnOffsetCommitRequest.json b/clients/src/main/resources/common/message/TxnOffsetCommitRequest.json index 0e7b187542..1df18c64b5 100644 --- a/clients/src/main/resources/common/message/TxnOffsetCommitRequest.json +++ b/clients/src/main/resources/common/message/TxnOffsetCommitRequest.json @@ -23,7 +23,9 @@ // Version 2 adds the committed leader epoch. // // Version 3 adds the member.id, group.instance.id and generation.id. - "validVersions": "0-3", + // + // Version 4 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId", diff --git a/clients/src/main/resources/common/message/TxnOffsetCommitResponse.json b/clients/src/main/resources/common/message/TxnOffsetCommitResponse.json index 96b03a078d..0f6c1f2724 100644 --- a/clients/src/main/resources/common/message/TxnOffsetCommitResponse.json +++ b/clients/src/main/resources/common/message/TxnOffsetCommitResponse.json @@ -22,7 +22,9 @@ // Version 2 is the same as version 1. // // Version 3 adds illegal generation, fenced instance id, and unknown member id errors. - "validVersions": "0-3", + // + // Version 4 adds support for new error code ABORTABLE_TRANSACTION (KIP-890). + "validVersions": "0-4", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", diff --git a/clients/src/test/java/org/apache/kafka/clients/CommonClientConfigsTest.java b/clients/src/test/java/org/apache/kafka/clients/CommonClientConfigsTest.java index c38f3e6a03..d53630f121 100644 --- a/clients/src/test/java/org/apache/kafka/clients/CommonClientConfigsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/CommonClientConfigsTest.java @@ -35,6 +35,7 @@ import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.ValidString.in; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -136,7 +137,7 @@ public void testMetricsReporters() { TestConfig config = new TestConfig(Collections.emptyMap()); List reporters = CommonClientConfigs.metricsReporters("clientId", config); assertEquals(1, reporters.size()); - assertTrue(reporters.get(0) instanceof JmxReporter); + assertInstanceOf(JmxReporter.class, reporters.get(0)); config = new TestConfig(Collections.singletonMap(CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_CONFIG, "false")); reporters = CommonClientConfigs.metricsReporters("clientId", config); @@ -145,7 +146,7 @@ public void testMetricsReporters() { config = new TestConfig(Collections.singletonMap(CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, JmxReporter.class.getName())); reporters = CommonClientConfigs.metricsReporters("clientId", config); assertEquals(1, reporters.size()); - assertTrue(reporters.get(0) instanceof JmxReporter); + assertInstanceOf(JmxReporter.class, reporters.get(0)); Map props = new HashMap<>(); props.put(CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, JmxReporter.class.getName() + "," + MyJmxReporter.class.getName()); 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 b8b3d54ef4..71e802365f 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 @@ -243,6 +243,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; @@ -270,6 +271,7 @@ import static org.apache.kafka.common.message.ListPartitionReassignmentsResponseData.OngoingTopicReassignment; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -293,7 +295,7 @@ public void testDefaultApiTimeoutAndRequestTimeoutConflicts() { final AdminClientConfig config = newConfMap(AdminClientConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, "500"); KafkaException exception = assertThrows(KafkaException.class, () -> KafkaAdminClient.createInternal(config, null)); - assertTrue(exception.getCause() instanceof ConfigException); + assertInstanceOf(ConfigException.class, exception.getCause()); } @Test @@ -498,7 +500,7 @@ public void testAdminClientFailureWhenClosed() { ExecutionException e = assertThrows(ExecutionException.class, () -> env.adminClient().createTopics( singleton(new NewTopic("myTopic", Collections.singletonMap(0, asList(0, 1, 2)))), new CreateTopicsOptions().timeoutMs(10000)).all().get()); - assertTrue(e.getCause() instanceof IllegalStateException, + assertInstanceOf(IllegalStateException.class, e.getCause(), "Expected an IllegalStateException error, but got " + Utils.stackTrace(e)); } @@ -1455,38 +1457,38 @@ private void callAdminClientApisAndExpectAnAuthenticationError(AdminClientUnitTe ExecutionException e = assertThrows(ExecutionException.class, () -> env.adminClient().createTopics( singleton(new NewTopic("myTopic", Collections.singletonMap(0, asList(0, 1, 2)))), new CreateTopicsOptions().timeoutMs(10000)).all().get()); - assertTrue(e.getCause() instanceof AuthenticationException, + assertInstanceOf(AuthenticationException.class, e.getCause(), "Expected an authentication error, but got " + Utils.stackTrace(e)); Map counts = new HashMap<>(); counts.put("my_topic", NewPartitions.increaseTo(3)); counts.put("other_topic", NewPartitions.increaseTo(3, asList(asList(2), asList(3)))); e = assertThrows(ExecutionException.class, () -> env.adminClient().createPartitions(counts).all().get()); - assertTrue(e.getCause() instanceof AuthenticationException, + assertInstanceOf(AuthenticationException.class, e.getCause(), "Expected an authentication error, but got " + Utils.stackTrace(e)); e = assertThrows(ExecutionException.class, () -> env.adminClient().createAcls(asList(ACL1, ACL2)).all().get()); - assertTrue(e.getCause() instanceof AuthenticationException, + assertInstanceOf(AuthenticationException.class, e.getCause(), "Expected an authentication error, but got " + Utils.stackTrace(e)); e = assertThrows(ExecutionException.class, () -> env.adminClient().describeAcls(FILTER1).values().get()); - assertTrue(e.getCause() instanceof AuthenticationException, + assertInstanceOf(AuthenticationException.class, e.getCause(), "Expected an authentication error, but got " + Utils.stackTrace(e)); e = assertThrows(ExecutionException.class, () -> env.adminClient().deleteAcls(asList(FILTER1, FILTER2)).all().get()); - assertTrue(e.getCause() instanceof AuthenticationException, + assertInstanceOf(AuthenticationException.class, e.getCause(), "Expected an authentication error, but got " + Utils.stackTrace(e)); e = assertThrows(ExecutionException.class, () -> env.adminClient().describeConfigs( singleton(new ConfigResource(ConfigResource.Type.BROKER, "0"))).all().get()); - assertTrue(e.getCause() instanceof AuthenticationException, + assertInstanceOf(AuthenticationException.class, e.getCause(), "Expected an authentication error, but got " + Utils.stackTrace(e)); } private void callClientQuotasApisAndExpectAnAuthenticationError(AdminClientUnitTestEnv env) { ExecutionException e = assertThrows(ExecutionException.class, () -> env.adminClient().describeClientQuotas(ClientQuotaFilter.all()).entities().get()); - assertTrue(e.getCause() instanceof AuthenticationException, + assertInstanceOf(AuthenticationException.class, e.getCause(), "Expected an authentication error, but got " + Utils.stackTrace(e)); ClientQuotaEntity entity = new ClientQuotaEntity(Collections.singletonMap(ClientQuotaEntity.USER, "user")); @@ -1494,7 +1496,7 @@ private void callClientQuotasApisAndExpectAnAuthenticationError(AdminClientUnitT e = assertThrows(ExecutionException.class, () -> env.adminClient().alterClientQuotas(asList(alteration)).all().get()); - assertTrue(e.getCause() instanceof AuthenticationException, + assertInstanceOf(AuthenticationException.class, e.getCause(), "Expected an authentication error, but got " + Utils.stackTrace(e)); } @@ -1876,7 +1878,7 @@ public void testDescribeLogDirs() throws ExecutionException, InterruptedExceptio env.cluster().nodeById(0)); final DescribeLogDirsResult errorResult = env.adminClient().describeLogDirs(brokers); ExecutionException exception = assertThrows(ExecutionException.class, () -> errorResult.allDescriptions().get()); - assertTrue(exception.getCause() instanceof ClusterAuthorizationException); + assertInstanceOf(ClusterAuthorizationException.class, exception.getCause()); // Empty results with an error with version >= 3 env.kafkaClient().prepareResponseFrom( @@ -1884,7 +1886,7 @@ public void testDescribeLogDirs() throws ExecutionException, InterruptedExceptio env.cluster().nodeById(0)); final DescribeLogDirsResult errorResult2 = env.adminClient().describeLogDirs(brokers); exception = assertThrows(ExecutionException.class, () -> errorResult2.allDescriptions().get()); - assertTrue(exception.getCause() instanceof UnknownServerException); + assertInstanceOf(UnknownServerException.class, exception.getCause()); } } @@ -1940,7 +1942,7 @@ public void testDescribeLogDirsWithVolumeBytes() throws ExecutionException, Inte env.cluster().nodeById(0)); final DescribeLogDirsResult errorResult = env.adminClient().describeLogDirs(brokers); ExecutionException exception = assertThrows(ExecutionException.class, () -> errorResult.allDescriptions().get()); - assertTrue(exception.getCause() instanceof ClusterAuthorizationException); + assertInstanceOf(ClusterAuthorizationException.class, exception.getCause()); // Empty results with an error with version >= 3 env.kafkaClient().prepareResponseFrom( @@ -1948,7 +1950,7 @@ public void testDescribeLogDirsWithVolumeBytes() throws ExecutionException, Inte env.cluster().nodeById(0)); final DescribeLogDirsResult errorResult2 = env.adminClient().describeLogDirs(brokers); exception = assertThrows(ExecutionException.class, () -> errorResult2.allDescriptions().get()); - assertTrue(exception.getCause() instanceof UnknownServerException); + assertInstanceOf(UnknownServerException.class, exception.getCause()); } } @@ -2167,14 +2169,9 @@ public void testCreatePartitions() throws Exception { KafkaFuture myTopicResult = values.get("my_topic"); myTopicResult.get(); KafkaFuture otherTopicResult = values.get("other_topic"); - try { - otherTopicResult.get(); - fail("get() should throw ExecutionException"); - } catch (ExecutionException e0) { - assertTrue(e0.getCause() instanceof InvalidTopicException); - InvalidTopicException e = (InvalidTopicException) e0.getCause(); - assertEquals("some detailed reason", e.getMessage()); - } + assertEquals("some detailed reason", + assertInstanceOf(InvalidTopicException.class, + assertThrows(ExecutionException.class, otherTopicResult::get).getCause()).getMessage()); } } @@ -2433,30 +2430,18 @@ public void testDeleteRecords() throws Exception { // "offset out of range" failure on records deletion for partition 1 KafkaFuture myTopicPartition1Result = values.get(myTopicPartition1); - try { - myTopicPartition1Result.get(); - fail("get() should throw ExecutionException"); - } catch (ExecutionException e0) { - assertTrue(e0.getCause() instanceof OffsetOutOfRangeException); - } + assertInstanceOf(OffsetOutOfRangeException.class, + assertThrows(ExecutionException.class, myTopicPartition1Result::get).getCause()); // not authorized to delete records for partition 2 KafkaFuture myTopicPartition2Result = values.get(myTopicPartition2); - try { - myTopicPartition2Result.get(); - fail("get() should throw ExecutionException"); - } catch (ExecutionException e1) { - assertTrue(e1.getCause() instanceof TopicAuthorizationException); - } + assertInstanceOf(TopicAuthorizationException.class, + assertThrows(ExecutionException.class, myTopicPartition2Result::get).getCause()); // the response does not contain a result for partition 3 KafkaFuture myTopicPartition3Result = values.get(myTopicPartition3); - try { - myTopicPartition3Result.get(); - fail("get() should throw ExecutionException"); - } catch (ExecutionException e1) { - assertTrue(e1.getCause() instanceof ApiException); - } + assertInstanceOf(ApiException.class, + assertThrows(ExecutionException.class, myTopicPartition3Result::get).getCause()); } } @@ -2811,6 +2796,68 @@ public void testListConsumerGroupsWithStates() throws Exception { } } + @Test + public void testListConsumerGroupsWithTypes() throws Exception { + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + // Test with a specific state filter but no type filter in list consumer group options. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.kafkaClient().prepareResponseFrom( + expectListGroupsRequestWithFilters(singleton(ConsumerGroupState.STABLE.toString()), Collections.emptySet()), + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(Arrays.asList( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable") + .setGroupType(GroupType.CLASSIC.toString())))), + env.cluster().nodeById(0)); + + final ListConsumerGroupsOptions options = new ListConsumerGroupsOptions().inStates(singleton(ConsumerGroupState.STABLE)); + final ListConsumerGroupsResult result = env.adminClient().listConsumerGroups(options); + Collection listings = result.valid().get(); + + assertEquals(1, listings.size()); + List expected = new ArrayList<>(); + expected.add(new ConsumerGroupListing("group-1", false, Optional.of(ConsumerGroupState.STABLE), Optional.of(GroupType.CLASSIC))); + assertEquals(expected, listings); + assertEquals(0, result.errors().get().size()); + + // Test with list consumer group options. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.kafkaClient().prepareResponseFrom( + expectListGroupsRequestWithFilters(Collections.emptySet(), singleton(GroupType.CONSUMER.toString())), + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(Arrays.asList( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable") + .setGroupType(GroupType.CONSUMER.toString()), + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-2") + .setGroupState("Empty") + .setGroupType(GroupType.CONSUMER.toString())))), + env.cluster().nodeById(0)); + + final ListConsumerGroupsOptions options2 = new ListConsumerGroupsOptions().withTypes(singleton(GroupType.CONSUMER)); + final ListConsumerGroupsResult result2 = env.adminClient().listConsumerGroups(options2); + Collection listings2 = result2.valid().get(); + + assertEquals(2, listings2.size()); + List expected2 = new ArrayList<>(); + expected2.add(new ConsumerGroupListing("group-2", true, Optional.of(ConsumerGroupState.EMPTY), Optional.of(GroupType.CONSUMER))); + expected2.add(new ConsumerGroupListing("group-1", false, Optional.of(ConsumerGroupState.STABLE), Optional.of(GroupType.CONSUMER))); + assertEquals(expected2, listings2); + assertEquals(0, result.errors().get().size()); + } + } + @Test public void testListConsumerGroupsWithStatesOlderBrokerVersion() throws Exception { ApiVersion listGroupV3 = new ApiVersion() @@ -2835,7 +2882,7 @@ public void testListConsumerGroupsWithStatesOlderBrokerVersion() throws Exceptio ListConsumerGroupsResult result = env.adminClient().listConsumerGroups(options); Collection listing = result.all().get(); assertEquals(1, listing.size()); - List expected = Collections.singletonList(new ConsumerGroupListing("group-1", false, Optional.empty())); + List expected = Collections.singletonList(new ConsumerGroupListing("group-1", false)); assertEquals(expected, listing); // But we cannot set a state filter with older broker @@ -2849,6 +2896,65 @@ public void testListConsumerGroupsWithStatesOlderBrokerVersion() throws Exceptio } } + @Test + public void testListConsumerGroupsWithTypesOlderBrokerVersion() throws Exception { + ApiVersion listGroupV4 = new ApiVersion() + .setApiKey(ApiKeys.LIST_GROUPS.id) + .setMinVersion((short) 0) + .setMaxVersion((short) 4); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(Collections.singletonList(listGroupV4))); + + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + // Check if we can list groups with older broker if we specify states and don't specify types. + env.kafkaClient().prepareResponseFrom( + expectListGroupsRequestWithFilters(singleton(ConsumerGroupState.STABLE.toString()), Collections.emptySet()), + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(Collections.singletonList( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState(ConsumerGroupState.STABLE.toString())))), + env.cluster().nodeById(0)); + + ListConsumerGroupsOptions options = new ListConsumerGroupsOptions().inStates(singleton(ConsumerGroupState.STABLE)); + ListConsumerGroupsResult result = env.adminClient().listConsumerGroups(options); + + Collection listing = result.all().get(); + assertEquals(1, listing.size()); + List expected = Collections.singletonList( + new ConsumerGroupListing("group-1", false, Optional.of(ConsumerGroupState.STABLE)) + ); + assertEquals(expected, listing); + + // Check that we cannot set a type filter with an older broker. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + env.kafkaClient().prepareUnsupportedVersionResponse(request -> + request instanceof ListGroupsRequest && !((ListGroupsRequest) request).data().typesFilter().isEmpty() + ); + + options = new ListConsumerGroupsOptions().withTypes(singleton(GroupType.CLASSIC)); + result = env.adminClient().listConsumerGroups(options); + TestUtils.assertFutureThrows(result.all(), UnsupportedVersionException.class); + } + } + + private MockClient.RequestMatcher expectListGroupsRequestWithFilters( + Set expectedStates, + Set expectedTypes + ) { + return body -> { + if (body instanceof ListGroupsRequest) { + ListGroupsRequest request = (ListGroupsRequest) body; + return Objects.equals(new HashSet<>(request.data().statesFilter()), expectedStates) + && Objects.equals(new HashSet<>(request.data().typesFilter()), expectedTypes); + } + return false; + }; + } + @Test public void testOffsetCommitNumRetries() throws Exception { final Cluster cluster = mockCluster(3, 0); @@ -4614,8 +4720,8 @@ public void testRemoveMembersFromGroup() throws Exception { new RemoveMembersFromConsumerGroupOptions() ); ExecutionException exception = assertThrows(ExecutionException.class, () -> partialFailureResults.all().get()); - assertTrue(exception.getCause() instanceof KafkaException); - assertTrue(exception.getCause().getCause() instanceof UnknownMemberIdException); + assertInstanceOf(KafkaException.class, exception.getCause()); + assertInstanceOf(UnknownMemberIdException.class, exception.getCause().getCause()); // Return with success for "removeAll" scenario // 1 prepare response for AdminClient.describeConsumerGroups diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandlerTest.java index 68c850583d..e0e4e9b202 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandlerTest.java @@ -38,9 +38,9 @@ import static java.util.Collections.singletonList; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; public class AbortTransactionHandlerTest { private final LogContext logContext = new LogContext(); @@ -214,7 +214,7 @@ private void assertFailed( assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptyList(), result.unmappedKeys); assertEquals(singleton(topicPartition), result.failedKeys.keySet()); - assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(topicPartition))); + assertInstanceOf(expectedExceptionType, result.failedKeys.get(topicPartition)); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategyTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategyTest.java index dd83c6be19..fc52e9e671 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategyTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategyTest.java @@ -37,8 +37,8 @@ import static java.util.Collections.singletonMap; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; public class CoordinatorStrategyTest { @@ -217,7 +217,7 @@ public void testFatalErrorOldLookupResponses() { assertFatalOldLookup(group, Errors.UNKNOWN_SERVER_ERROR); Throwable throwable = assertFatalOldLookup(group, Errors.GROUP_AUTHORIZATION_FAILED); - assertTrue(throwable instanceof GroupAuthorizationException); + assertInstanceOf(GroupAuthorizationException.class, throwable); GroupAuthorizationException exception = (GroupAuthorizationException) throwable; assertEquals("foo", exception.groupId()); } @@ -233,7 +233,7 @@ public Throwable assertFatalOldLookup( assertEquals(singleton(key), result.failedKeys.keySet()); Throwable throwable = result.failedKeys.get(key); - assertTrue(error.exception().getClass().isInstance(throwable)); + assertInstanceOf(error.exception().getClass(), throwable); return throwable; } @@ -244,7 +244,7 @@ public void testFatalErrorLookupResponses() { assertFatalLookup(group, Errors.UNKNOWN_SERVER_ERROR); Throwable throwable = assertFatalLookup(group, Errors.GROUP_AUTHORIZATION_FAILED); - assertTrue(throwable instanceof GroupAuthorizationException); + assertInstanceOf(GroupAuthorizationException.class, throwable); GroupAuthorizationException exception = (GroupAuthorizationException) throwable; assertEquals("foo", exception.groupId()); } @@ -264,7 +264,7 @@ public Throwable assertFatalLookup( assertEquals(singleton(key), result.failedKeys.keySet()); Throwable throwable = result.failedKeys.get(key); - assertTrue(error.exception().getClass().isInstance(throwable)); + assertInstanceOf(error.exception().getClass(), throwable); return throwable; } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupOffsetsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupOffsetsHandlerTest.java index 629ca89801..fef8a55074 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupOffsetsHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupOffsetsHandlerTest.java @@ -21,7 +21,7 @@ import static java.util.Collections.singleton; import static java.util.Collections.singletonList; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import java.util.Arrays; import java.util.Collection; @@ -189,7 +189,7 @@ private void assertGroupFailed( assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptyList(), result.unmappedKeys); assertEquals(singleton(key), result.failedKeys.keySet()); - assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); + assertInstanceOf(expectedExceptionType, result.failedKeys.get(key)); } private void assertPartitionFailed( diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupsHandlerTest.java index 3e7cead6f5..1d1b152afb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupsHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteConsumerGroupsHandlerTest.java @@ -21,7 +21,7 @@ import static java.util.Collections.singleton; import static java.util.Collections.singletonList; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.GroupAuthorizationException; @@ -125,6 +125,6 @@ private void assertFailed( assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptyList(), result.unmappedKeys); assertEquals(singleton(key), result.failedKeys.keySet()); - assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); + assertInstanceOf(expectedExceptionType, result.failedKeys.get(key)); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandlerTest.java index c39747f1fb..58492696c4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DeleteRecordsHandlerTest.java @@ -22,15 +22,24 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; + import org.apache.kafka.clients.admin.DeletedRecords; import org.apache.kafka.clients.admin.RecordsToDelete; +import org.apache.kafka.clients.admin.internals.AdminApiLookupStrategy.LookupResult; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.DeleteRecordsRequestData; import org.apache.kafka.common.message.DeleteRecordsResponseData; +import org.apache.kafka.common.message.MetadataResponseData; +import org.apache.kafka.common.message.MetadataResponseData.MetadataResponsePartition; +import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.DeleteRecordsRequest; import org.apache.kafka.common.requests.DeleteRecordsResponse; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.utils.LogContext; import org.junit.jupiter.api.Test; @@ -41,6 +50,7 @@ import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertTrue; public class DeleteRecordsHandlerTest { @@ -50,7 +60,8 @@ public class DeleteRecordsHandlerTest { private final TopicPartition t0p1 = new TopicPartition("t0", 1); private final TopicPartition t0p2 = new TopicPartition("t0", 2); private final TopicPartition t0p3 = new TopicPartition("t0", 3); - private final Node node = new Node(1, "host", 1234); + private final Node node1 = new Node(1, "host", 1234); + private final Node node2 = new Node(2, "host", 1235); private final Map recordsToDelete = new HashMap() { { put(t0p0, RecordsToDelete.beforeOffset(10L)); @@ -63,11 +74,11 @@ public class DeleteRecordsHandlerTest { @Test public void testBuildRequestSimple() { DeleteRecordsHandler handler = new DeleteRecordsHandler(recordsToDelete, logContext, timeout); - DeleteRecordsRequest request = handler.buildBatchedRequest(node.id(), mkSet(t0p0, t0p1)).build(); - List topicPartitions = request.data().topics(); - assertEquals(1, topicPartitions.size()); - DeleteRecordsRequestData.DeleteRecordsTopic topic = topicPartitions.get(0); - assertEquals(4, topic.partitions().size()); + DeleteRecordsRequest request = handler.buildBatchedRequest(node1.id(), mkSet(t0p0, t0p1)).build(); + List topics = request.data().topics(); + assertEquals(1, topics.size()); + DeleteRecordsRequestData.DeleteRecordsTopic topic = topics.get(0); + assertEquals(2, topic.partitions().size()); } @Test @@ -199,6 +210,54 @@ public void testHandleResponseSanityCheck() { assertTrue(result.unmappedKeys.isEmpty()); } + // This is a more complicated test which ensures that DeleteRecords requests for multiple + // leader nodes are correctly divided up among the nodes based on leadership. + // node1 leads t0p0 and t0p2, while node2 leads t0p1 and t0p3. + @Test + public void testBuildRequestMultipleLeaders() { + MetadataResponseData metadataResponseData = new MetadataResponseData(); + MetadataResponseTopic topicMetadata = new MetadataResponseTopic(); + topicMetadata.setName("t0").setErrorCode(Errors.NONE.code()); + topicMetadata.partitions().add(new MetadataResponsePartition() + .setPartitionIndex(0).setLeaderId(node1.id()).setErrorCode(Errors.NONE.code())); + topicMetadata.partitions().add(new MetadataResponsePartition() + .setPartitionIndex(1).setLeaderId(node2.id()).setErrorCode(Errors.NONE.code())); + topicMetadata.partitions().add(new MetadataResponsePartition() + .setPartitionIndex(2).setLeaderId(node1.id()).setErrorCode(Errors.NONE.code())); + topicMetadata.partitions().add(new MetadataResponsePartition() + .setPartitionIndex(3).setLeaderId(node2.id()).setErrorCode(Errors.NONE.code())); + metadataResponseData.topics().add(topicMetadata); + MetadataResponse metadataResponse = new MetadataResponse(metadataResponseData, ApiKeys.METADATA.latestVersion()); + + DeleteRecordsHandler handler = new DeleteRecordsHandler(recordsToDelete, logContext, timeout); + AdminApiLookupStrategy strategy = handler.lookupStrategy(); + assertInstanceOf(PartitionLeaderStrategy.class, strategy); + PartitionLeaderStrategy specificStrategy = (PartitionLeaderStrategy) strategy; + MetadataRequest request = specificStrategy.buildRequest(mkSet(t0p0, t0p1, t0p2, t0p3)).build(); + assertEquals(mkSet("t0"), new HashSet<>(request.topics())); + + Set tpSet = mkSet(t0p0, t0p1, t0p2, t0p3); + LookupResult lookupResult = strategy.handleResponse(tpSet, metadataResponse); + assertEquals(emptyMap(), lookupResult.failedKeys); + assertEquals(tpSet, lookupResult.mappedKeys.keySet()); + + Map> partitionsPerBroker = new HashMap<>(); + lookupResult.mappedKeys.forEach((tp, node) -> partitionsPerBroker.computeIfAbsent(node, key -> new HashSet<>()).add(tp)); + + DeleteRecordsRequest deleteRequest = handler.buildBatchedRequest(node1.id(), partitionsPerBroker.get(node1.id())).build(); + assertEquals(2, deleteRequest.data().topics().get(0).partitions().size()); + assertEquals(mkSet(t0p0, t0p2), + deleteRequest.data().topics().get(0).partitions().stream() + .map(drp -> new TopicPartition("t0", drp.partitionIndex())) + .collect(Collectors.toSet())); + deleteRequest = handler.buildBatchedRequest(node2.id(), partitionsPerBroker.get(node2.id())).build(); + assertEquals(2, deleteRequest.data().topics().get(0).partitions().size()); + assertEquals(mkSet(t0p1, t0p3), + deleteRequest.data().topics().get(0).partitions().stream() + .map(drp -> new TopicPartition("t0", drp.partitionIndex())) + .collect(Collectors.toSet())); + } + private DeleteRecordsResponse createResponse(Map errorsByPartition) { return createResponse(errorsByPartition, recordsToDelete.keySet()); } @@ -227,7 +286,7 @@ private DeleteRecordsResponse createResponse( private AdminApiHandler.ApiResult handleResponse(DeleteRecordsResponse response) { DeleteRecordsHandler handler = new DeleteRecordsHandler(recordsToDelete, logContext, timeout); - return handler.handleResponse(node, recordsToDelete.keySet(), response); + return handler.handleResponse(node1, recordsToDelete.keySet(), response); } private void assertResult( diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeConsumerGroupsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeConsumerGroupsHandlerTest.java index 0ece976404..7179e13a4f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeConsumerGroupsHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeConsumerGroupsHandlerTest.java @@ -22,8 +22,8 @@ import static java.util.Collections.singletonList; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.ArrayList; import java.util.Arrays; @@ -369,7 +369,7 @@ private void assertFailed( assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptyList(), result.unmappedKeys); assertEquals(singleton(key), result.failedKeys.keySet()); - assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); + assertInstanceOf(expectedExceptionType, result.failedKeys.get(key)); } private void assertRequestAndKeys( diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandlerTest.java index 0f39b4dd01..6ecf328819 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandlerTest.java @@ -52,8 +52,8 @@ import static java.util.Collections.singletonMap; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertTrue; public class DescribeProducersHandlerTest { private DescribeProducersHandler newHandler( @@ -137,7 +137,7 @@ public void testBuildRequest() { public void testAuthorizationFailure() { TopicPartition topicPartition = new TopicPartition("foo", 5); Throwable exception = assertFatalError(topicPartition, Errors.TOPIC_AUTHORIZATION_FAILED); - assertTrue(exception instanceof TopicAuthorizationException); + assertInstanceOf(TopicAuthorizationException.class, exception); TopicAuthorizationException authException = (TopicAuthorizationException) exception; assertEquals(mkSet("foo"), authException.unauthorizedTopics()); } @@ -146,7 +146,7 @@ public void testAuthorizationFailure() { public void testInvalidTopic() { TopicPartition topicPartition = new TopicPartition("foo", 5); Throwable exception = assertFatalError(topicPartition, Errors.INVALID_TOPIC_EXCEPTION); - assertTrue(exception instanceof InvalidTopicException); + assertInstanceOf(InvalidTopicException.class, exception); InvalidTopicException invalidTopicException = (InvalidTopicException) exception; assertEquals(mkSet("foo"), invalidTopicException.invalidTopics()); } @@ -155,7 +155,7 @@ public void testInvalidTopic() { public void testUnexpectedError() { TopicPartition topicPartition = new TopicPartition("foo", 5); Throwable exception = assertFatalError(topicPartition, Errors.UNKNOWN_SERVER_ERROR); - assertTrue(exception instanceof UnknownServerException); + assertInstanceOf(UnknownServerException.class, exception); } @Test @@ -185,7 +185,7 @@ public void testFatalNotLeaderErrorIfStaticMapped() { assertEquals(emptyList(), result.unmappedKeys); assertEquals(mkSet(topicPartition), result.failedKeys.keySet()); Throwable exception = result.failedKeys.get(topicPartition); - assertTrue(exception instanceof NotLeaderOrFollowerException); + assertInstanceOf(NotLeaderOrFollowerException.class, exception); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandlerTest.java index 7ffda2b00e..b744795cd9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandlerTest.java @@ -37,7 +37,7 @@ import static java.util.Collections.singletonList; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; public class DescribeTransactionsHandlerTest { private final LogContext logContext = new LogContext(); @@ -107,7 +107,7 @@ private void assertFatalError( assertEquals(mkSet(key), result.failedKeys.keySet()); Throwable throwable = result.failedKeys.get(key); - assertTrue(error.exception().getClass().isInstance(throwable)); + assertInstanceOf(error.exception().getClass(), throwable); } private void assertRetriableError( diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java index c4151ebb0e..8b20f84c98 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/FenceProducersHandlerTest.java @@ -34,7 +34,7 @@ import static java.util.Collections.singletonList; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; public class FenceProducersHandlerTest { private final LogContext logContext = new LogContext(); @@ -96,7 +96,7 @@ private void assertFatalError( assertEquals(mkSet(key), result.failedKeys.keySet()); Throwable throwable = result.failedKeys.get(key); - assertTrue(error.exception().getClass().isInstance(throwable)); + assertInstanceOf(error.exception().getClass(), throwable); } private void assertRetriableError( diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java index 95fabb3fc2..6484b842dd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java @@ -21,6 +21,7 @@ import static java.util.Collections.singleton; import static java.util.Collections.singletonList; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.Arrays; @@ -403,7 +404,7 @@ private void assertFailed( assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptyList(), result.unmappedKeys); assertEquals(singleton(key), result.failedKeys.keySet()); - assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); + assertInstanceOf(expectedExceptionType, result.failedKeys.get(key)); } private void assertFailedForMultipleGroups( @@ -415,7 +416,7 @@ private void assertFailedForMultipleGroups( for (String g : groupToExceptionMap.keySet()) { CoordinatorKey key = CoordinatorKey.byGroupId(g); assertTrue(result.failedKeys.containsKey(key)); - assertTrue(groupToExceptionMap.get(g).isInstance(result.failedKeys.get(key))); + assertInstanceOf(groupToExceptionMap.get(g), result.failedKeys.get(key)); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/PartitionLeaderStrategyTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/PartitionLeaderStrategyTest.java index f65b97b445..d17443b8be 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/PartitionLeaderStrategyTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/PartitionLeaderStrategyTest.java @@ -44,7 +44,7 @@ import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; public class PartitionLeaderStrategyTest { @@ -78,7 +78,7 @@ public void testBuildLookupRequest() { public void testTopicAuthorizationFailure() { TopicPartition topicPartition = new TopicPartition("foo", 0); Throwable exception = assertFatalTopicError(topicPartition, Errors.TOPIC_AUTHORIZATION_FAILED); - assertTrue(exception instanceof TopicAuthorizationException); + assertInstanceOf(TopicAuthorizationException.class, exception); TopicAuthorizationException authException = (TopicAuthorizationException) exception; assertEquals(mkSet("foo"), authException.unauthorizedTopics()); } @@ -87,7 +87,7 @@ public void testTopicAuthorizationFailure() { public void testInvalidTopicError() { TopicPartition topicPartition = new TopicPartition("foo", 0); Throwable exception = assertFatalTopicError(topicPartition, Errors.INVALID_TOPIC_EXCEPTION); - assertTrue(exception instanceof InvalidTopicException); + assertInstanceOf(InvalidTopicException.class, exception); InvalidTopicException invalidTopicException = (InvalidTopicException) exception; assertEquals(mkSet("foo"), invalidTopicException.invalidTopics()); } @@ -96,7 +96,7 @@ public void testInvalidTopicError() { public void testUnexpectedTopicError() { TopicPartition topicPartition = new TopicPartition("foo", 0); Throwable exception = assertFatalTopicError(topicPartition, Errors.UNKNOWN_SERVER_ERROR); - assertTrue(exception instanceof UnknownServerException); + assertInstanceOf(UnknownServerException.class, exception); } @Test @@ -121,7 +121,7 @@ public void testRetriablePartitionErrors() { public void testUnexpectedPartitionError() { TopicPartition topicPartition = new TopicPartition("foo", 0); Throwable exception = assertFatalPartitionError(topicPartition, Errors.UNKNOWN_SERVER_ERROR); - assertTrue(exception instanceof UnknownServerException); + assertInstanceOf(UnknownServerException.class, exception); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/RemoveMembersFromConsumerGroupHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/RemoveMembersFromConsumerGroupHandlerTest.java index 3ecd1f10ee..1ad66bf782 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/RemoveMembersFromConsumerGroupHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/RemoveMembersFromConsumerGroupHandlerTest.java @@ -21,7 +21,7 @@ import static java.util.Collections.singleton; import static java.util.Collections.singletonList; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import java.util.Arrays; import java.util.Collections; @@ -164,7 +164,7 @@ private void assertFailed( assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptyList(), result.unmappedKeys); assertEquals(singleton(key), result.failedKeys.keySet()); - assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); + assertInstanceOf(expectedExceptionType, result.failedKeys.get(key)); } private void assertMemberFailed( diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java index 6cbda7c299..3cab19a6a1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java @@ -35,7 +35,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; public class ConsumerPartitionAssignorTest { @@ -45,7 +44,7 @@ public void shouldInstantiateAssignor() { Collections.singletonList(StickyAssignor.class.getName()), Collections.emptyMap() ); - assertTrue(assignors.get(0) instanceof StickyAssignor); + assertInstanceOf(StickyAssignor.class, assignors.get(0)); } @Test @@ -54,8 +53,8 @@ public void shouldInstantiateListOfAssignors() { Arrays.asList(StickyAssignor.class.getName(), CooperativeStickyAssignor.class.getName()), Collections.emptyMap() ); - assertTrue(assignors.get(0) instanceof StickyAssignor); - assertTrue(assignors.get(1) instanceof CooperativeStickyAssignor); + assertInstanceOf(StickyAssignor.class, assignors.get(0)); + assertInstanceOf(CooperativeStickyAssignor.class, assignors.get(1)); } @Test @@ -80,7 +79,7 @@ public void shouldInstantiateFromClassType() { initConsumerConfigWithClassTypes(Collections.singletonList(StickyAssignor.class)) .getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG); List assignors = getAssignorInstances(classTypes, Collections.emptyMap()); - assertTrue(assignors.get(0) instanceof StickyAssignor); + assertInstanceOf(StickyAssignor.class, assignors.get(0)); } @Test @@ -91,8 +90,8 @@ public void shouldInstantiateFromListOfClassTypes() { List assignors = getAssignorInstances(classTypes, Collections.emptyMap()); - assertTrue(assignors.get(0) instanceof StickyAssignor); - assertTrue(assignors.get(1) instanceof CooperativeStickyAssignor); + assertInstanceOf(StickyAssignor.class, assignors.get(0)); + assertInstanceOf(CooperativeStickyAssignor.class, assignors.get(1)); } @Test 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 7dec7305d4..8cce3fb847 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 @@ -145,6 +145,7 @@ import static org.apache.kafka.common.utils.Utils.propsToMap; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotSame; @@ -259,7 +260,7 @@ public void testExplicitlyOnlyEnableJmxReporter(GroupProtocol groupProtocol) { props.setProperty(ConsumerConfig.ENABLE_METRICS_PUSH_CONFIG, "false"); consumer = newConsumer(props, new StringDeserializer(), new StringDeserializer()); assertEquals(1, consumer.metricsRegistry().reporters().size()); - assertTrue(consumer.metricsRegistry().reporters().get(0) instanceof JmxReporter); + assertInstanceOf(JmxReporter.class, consumer.metricsRegistry().reporters().get(0)); } @ParameterizedTest @@ -272,7 +273,7 @@ public void testExplicitlyOnlyEnableClientTelemetryReporter(GroupProtocol groupP props.setProperty(ConsumerConfig.AUTO_INCLUDE_JMX_REPORTER_CONFIG, "false"); consumer = newConsumer(props, new StringDeserializer(), new StringDeserializer()); assertEquals(1, consumer.metricsRegistry().reporters().size()); - assertTrue(consumer.metricsRegistry().reporters().get(0) instanceof ClientTelemetryReporter); + assertInstanceOf(ClientTelemetryReporter.class, consumer.metricsRegistry().reporters().get(0)); } // TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol. @@ -2071,7 +2072,7 @@ private void consumerCloseTest(GroupProtocol groupProtocol, TestUtils.waitForCondition( () -> closeException.get() != null, "InterruptException did not occur within timeout."); - assertTrue(closeException.get() instanceof InterruptException, "Expected exception not thrown " + closeException); + assertInstanceOf(InterruptException.class, closeException.get(), "Expected exception not thrown " + closeException); } else { future.get(closeTimeoutMs, TimeUnit.MILLISECONDS); // Should succeed without TimeoutException or ExecutionException assertNull(closeException.get(), "Unexpected exception during close"); 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 bfcf654767..1f13aea21f 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 @@ -78,6 +78,7 @@ import static java.util.Collections.emptyMap; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotSame; import static org.junit.jupiter.api.Assertions.assertSame; @@ -341,7 +342,7 @@ public void testGroupMaxSizeExceptionIsFatal() { RequestFuture future = coordinator.sendJoinGroupRequest(); assertTrue(consumerClient.poll(future, mockTime.timer(REQUEST_TIMEOUT_MS))); - assertTrue(future.exception().getClass().isInstance(Errors.GROUP_MAX_SIZE_REACHED.exception())); + assertInstanceOf(future.exception().getClass(), Errors.GROUP_MAX_SIZE_REACHED.exception()); assertFalse(future.isRetriable()); } @@ -359,7 +360,7 @@ public void testJoinGroupRequestTimeout() { mockTime.sleep(REBALANCE_TIMEOUT_MS - REQUEST_TIMEOUT_MS + AbstractCoordinator.JOIN_GROUP_TIMEOUT_LAPSE); assertTrue(consumerClient.poll(future, mockTime.timer(0))); - assertTrue(future.exception() instanceof DisconnectException); + assertInstanceOf(DisconnectException.class, future.exception()); } @Test @@ -377,7 +378,7 @@ public void testJoinGroupRequestTimeoutLowerBoundedByDefaultRequestTimeout() { mockTime.sleep(expectedRequestDeadline - mockTime.milliseconds() + 1); assertTrue(consumerClient.poll(future, mockTime.timer(0))); - assertTrue(future.exception() instanceof DisconnectException); + assertInstanceOf(DisconnectException.class, future.exception()); } @Test @@ -773,7 +774,7 @@ public void testJoinGroupUnknownMemberResponseWithOldGeneration() throws Interru mockClient.respond(joinGroupFollowerResponse(currGen.generationId + 1, memberId, JoinGroupRequest.UNKNOWN_MEMBER_ID, Errors.UNKNOWN_MEMBER_ID)); assertTrue(consumerClient.poll(future, mockTime.timer(REQUEST_TIMEOUT_MS))); - assertTrue(future.exception().getClass().isInstance(Errors.UNKNOWN_MEMBER_ID.exception())); + assertInstanceOf(future.exception().getClass(), Errors.UNKNOWN_MEMBER_ID.exception()); // the generation should not be reset assertEquals(newGen, coordinator.generation()); @@ -813,7 +814,7 @@ public void testSyncGroupUnknownMemberResponseWithOldGeneration() throws Interru mockClient.respond(syncGroupResponse(Errors.UNKNOWN_MEMBER_ID)); assertTrue(consumerClient.poll(future, mockTime.timer(REQUEST_TIMEOUT_MS))); - assertTrue(future.exception().getClass().isInstance(Errors.UNKNOWN_MEMBER_ID.exception())); + assertInstanceOf(future.exception().getClass(), Errors.UNKNOWN_MEMBER_ID.exception()); // the generation should not be reset assertEquals(newGen, coordinator.generation()); @@ -853,7 +854,7 @@ public void testSyncGroupIllegalGenerationResponseWithOldGeneration() throws Int mockClient.respond(syncGroupResponse(Errors.ILLEGAL_GENERATION)); assertTrue(consumerClient.poll(future, mockTime.timer(REQUEST_TIMEOUT_MS))); - assertTrue(future.exception().getClass().isInstance(Errors.ILLEGAL_GENERATION.exception())); + assertInstanceOf(future.exception().getClass(), Errors.ILLEGAL_GENERATION.exception()); // the generation should not be reset assertEquals(newGen, coordinator.generation()); @@ -1027,18 +1028,17 @@ public void testHeartbeatRequestWithFencedInstanceIdException() throws Interrupt mockClient.prepareResponse(syncGroupResponse(Errors.NONE)); mockClient.prepareResponse(heartbeatResponse(Errors.FENCED_INSTANCE_ID)); - try { - coordinator.ensureActiveGroup(); - mockTime.sleep(HEARTBEAT_INTERVAL_MS); - long startMs = System.currentTimeMillis(); - while (System.currentTimeMillis() - startMs < 1000) { - Thread.sleep(10); - coordinator.pollHeartbeat(mockTime.milliseconds()); - } - fail("Expected pollHeartbeat to raise fenced instance id exception in 1 second"); - } catch (RuntimeException exception) { - assertTrue(exception instanceof FencedInstanceIdException); - } + assertThrows(FencedInstanceIdException.class, + () -> { + coordinator.ensureActiveGroup(); + mockTime.sleep(HEARTBEAT_INTERVAL_MS); + long startMs = System.currentTimeMillis(); + while (System.currentTimeMillis() - startMs < 1000) { + Thread.sleep(10); + coordinator.pollHeartbeat(mockTime.milliseconds()); + } + }, + "Expected pollHeartbeat to raise fenced instance id exception in 1 second"); } @Test @@ -1069,7 +1069,7 @@ public void testJoinGroupRequestWithRebalanceInProgress() { RequestFuture future = coordinator.sendJoinGroupRequest(); assertTrue(consumerClient.poll(future, mockTime.timer(REQUEST_TIMEOUT_MS))); - assertTrue(future.exception().getClass().isInstance(Errors.REBALANCE_IN_PROGRESS.exception())); + assertInstanceOf(future.exception().getClass(), Errors.REBALANCE_IN_PROGRESS.exception()); assertEquals(Errors.REBALANCE_IN_PROGRESS.message(), future.exception().getMessage()); assertTrue(coordinator.rejoinNeededOrPending()); @@ -1157,7 +1157,7 @@ public void testHandleMultipleMembersLeaveGroupResponse() { leaveGroupResponse(Arrays.asList(memberResponse, memberResponse)); RequestFuture leaveGroupFuture = setupLeaveGroup(response); assertNotNull(leaveGroupFuture); - assertTrue(leaveGroupFuture.exception() instanceof IllegalStateException); + assertInstanceOf(IllegalStateException.class, leaveGroupFuture.exception()); } @Test @@ -1178,7 +1178,7 @@ public void testHandleLeaveGroupResponseWithException() { leaveGroupResponse(Collections.singletonList(memberResponse)); RequestFuture leaveGroupFuture = setupLeaveGroup(response); assertNotNull(leaveGroupFuture); - assertTrue(leaveGroupFuture.exception() instanceof UnknownMemberIdException); + assertInstanceOf(UnknownMemberIdException.class, leaveGroupFuture.exception()); } private RequestFuture setupLeaveGroup(LeaveGroupResponse leaveGroupResponse) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 976677dec8..fa413eed36 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -30,32 +30,32 @@ import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.consumer.RetriableCommitFailedException; import org.apache.kafka.clients.consumer.RoundRobinAssignor; +import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; -import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.AsyncCommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; +import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; -import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.clients.consumer.internals.events.EventProcessor; -import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; -import org.apache.kafka.clients.consumer.internals.events.LeaveOnCloseApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.FetchCommittedOffsetsEvent; +import org.apache.kafka.clients.consumer.internals.events.LeaveOnCloseEvent; +import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; -import org.apache.kafka.clients.consumer.internals.events.PollApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.SyncCommitApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.UnsubscribeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; -import org.apache.kafka.common.Cluster; +import org.apache.kafka.clients.consumer.internals.events.PollEvent; +import org.apache.kafka.clients.consumer.internals.events.ResetPositionsEvent; +import org.apache.kafka.clients.consumer.internals.events.SubscriptionChangeEvent; +import org.apache.kafka.clients.consumer.internals.events.SyncCommitEvent; +import org.apache.kafka.clients.consumer.internals.events.UnsubscribeEvent; +import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.FencedInstanceIdException; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.RetriableException; @@ -79,6 +79,7 @@ import org.junit.jupiter.params.provider.MethodSource; import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatchers; +import org.mockito.MockedStatic; import org.mockito.Mockito; import java.time.Duration; @@ -101,7 +102,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -116,10 +116,13 @@ import static org.apache.kafka.clients.consumer.internals.MembershipManagerImpl.TOPIC_PARTITION_COMPARATOR; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.test.TestUtils.requiredConsumerConfig; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -131,6 +134,7 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -162,19 +166,19 @@ public void resetAll() { } private AsyncKafkaConsumer newConsumer() { - final Properties props = requiredConsumerProperties(); + final Properties props = requiredConsumerConfig(); props.put(ConsumerConfig.GROUP_ID_CONFIG, "group-id"); return newConsumer(props); } private AsyncKafkaConsumer newConsumerWithoutGroupId() { - final Properties props = requiredConsumerProperties(); + final Properties props = requiredConsumerConfig(); return newConsumer(props); } @SuppressWarnings("UnusedReturnValue") private AsyncKafkaConsumer newConsumerWithEmptyGroupId() { - final Properties props = requiredConsumerPropertiesAndGroupId(""); + final Properties props = requiredConsumerConfigAndGroupId(""); return newConsumer(props); } @@ -256,9 +260,9 @@ public void testCommitAsyncWithNullCallback() { consumer.commitAsync(offsets, null); - final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(AsyncCommitApplicationEvent.class); + final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(AsyncCommitEvent.class); verify(applicationEventHandler).add(commitEventCaptor.capture()); - final AsyncCommitApplicationEvent commitEvent = commitEventCaptor.getValue(); + final AsyncCommitEvent commitEvent = commitEventCaptor.getValue(); assertEquals(offsets, commitEvent.offsets()); assertDoesNotThrow(() -> commitEvent.future().complete(null)); assertDoesNotThrow(() -> consumer.commitAsync(offsets, null)); @@ -310,9 +314,9 @@ public void testCommitAsyncWithFencedException() { assertDoesNotThrow(() -> consumer.commitAsync(offsets, callback)); - final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(AsyncCommitApplicationEvent.class); + final ArgumentCaptor commitEventCaptor = ArgumentCaptor.forClass(AsyncCommitEvent.class); verify(applicationEventHandler).add(commitEventCaptor.capture()); - final AsyncCommitApplicationEvent commitEvent = commitEventCaptor.getValue(); + final AsyncCommitEvent commitEvent = commitEventCaptor.getValue(); commitEvent.future().completeExceptionally(Errors.FENCED_INSTANCE_ID.exception()); assertThrows(Errors.FENCED_INSTANCE_ID.exception().getClass(), () -> consumer.commitAsync()); @@ -325,7 +329,7 @@ public void testCommitted() { completeFetchedCommittedOffsetApplicationEventSuccessfully(topicPartitionOffsets); assertEquals(topicPartitionOffsets, consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), any()); final Metric metric = consumer.metrics() .get(consumer.metricsRegistry().metricName("committed-time-ns-total", "consumer-metrics")); assertTrue((double) metric.metricValue() > 0); @@ -347,7 +351,7 @@ public void testCommittedLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t2, 3); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), any()); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), any()); } @Test @@ -355,9 +359,9 @@ public void testCommittedExceptionThrown() { consumer = newConsumer(); Map offsets = mockTopicPartitionOffset(); when(applicationEventHandler.addAndGet( - any(FetchCommittedOffsetsApplicationEvent.class), any())).thenAnswer(invocation -> { + any(FetchCommittedOffsetsEvent.class), any())).thenAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); - assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); + assertInstanceOf(FetchCommittedOffsetsEvent.class, event); throw new KafkaException("Test exception"); }); @@ -530,7 +534,7 @@ public void testCommitSyncLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t1, 1); - verify(applicationEventHandler).add(ArgumentMatchers.isA(SyncCommitApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(SyncCommitEvent.class)); } @Test @@ -564,7 +568,73 @@ public void testCommitAsyncLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t1, 1); - verify(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitEvent.class)); + } + + @Test + public void testCommitAsyncTriggersFencedExceptionFromCommitAsync() { + final String groupId = "consumerGroupA"; + final String groupInstanceId = "groupInstanceId1"; + final Properties props = requiredConsumerConfigAndGroupId(groupId); + props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = newConsumer(config); + completeCommitAsyncApplicationEventExceptionally(Errors.FENCED_INSTANCE_ID.exception()); + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + completeFetchedCommittedOffsetApplicationEventSuccessfully(mkMap()); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + final TopicPartition tp = new TopicPartition("foo", 0); + consumer.assign(Collections.singleton(tp)); + consumer.seek(tp, 20); + + assertDoesNotThrow(() -> consumer.commitAsync()); + + Exception e = assertThrows(FencedInstanceIdException.class, () -> consumer.commitAsync()); + assertEquals("Get fenced exception for group.instance.id groupInstanceId1", e.getMessage()); + } + + @Test + public void testCommitSyncTriggersFencedExceptionFromCommitAsync() { + final String groupId = "consumerGroupA"; + final String groupInstanceId = "groupInstanceId1"; + final Properties props = requiredConsumerConfigAndGroupId(groupId); + props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = newConsumer(config); + completeCommitAsyncApplicationEventExceptionally(Errors.FENCED_INSTANCE_ID.exception()); + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + completeFetchedCommittedOffsetApplicationEventSuccessfully(mkMap()); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + final TopicPartition tp = new TopicPartition("foo", 0); + consumer.assign(Collections.singleton(tp)); + consumer.seek(tp, 20); + + assertDoesNotThrow(() -> consumer.commitAsync()); + + Exception e = assertThrows(FencedInstanceIdException.class, () -> consumer.commitSync()); + assertEquals("Get fenced exception for group.instance.id groupInstanceId1", e.getMessage()); + } + + @Test + public void testPollTriggersFencedExceptionFromCommitAsync() { + final String groupId = "consumerGroupA"; + final String groupInstanceId = "groupInstanceId1"; + final Properties props = requiredConsumerConfigAndGroupId(groupId); + props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId); + final ConsumerConfig config = new ConsumerConfig(props); + consumer = newConsumer(config); + completeCommitAsyncApplicationEventExceptionally(Errors.FENCED_INSTANCE_ID.exception()); + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + completeFetchedCommittedOffsetApplicationEventSuccessfully(mkMap()); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + final TopicPartition tp = new TopicPartition("foo", 0); + consumer.assign(Collections.singleton(tp)); + consumer.seek(tp, 20); + + assertDoesNotThrow(() -> consumer.commitAsync()); + + Exception e = assertThrows(FencedInstanceIdException.class, () -> consumer.poll(Duration.ZERO)); + assertEquals("Get fenced exception for group.instance.id groupInstanceId1", e.getMessage()); } @Test @@ -598,8 +668,8 @@ public void testVerifyApplicationEventOnShutdown() { consumer = newConsumer(); doReturn(null).when(applicationEventHandler).addAndGet(any(), any()); consumer.close(); - verify(applicationEventHandler).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); - verify(applicationEventHandler).add(any(CommitOnCloseApplicationEvent.class)); + verify(applicationEventHandler).addAndGet(any(LeaveOnCloseEvent.class), any()); + verify(applicationEventHandler).add(any(CommitOnCloseEvent.class)); } @Test @@ -641,7 +711,7 @@ public void testFailedPartitionRevocationOnClose() { subscriptions.assignFromSubscribed(singleton(tp)); doThrow(new KafkaException()).when(listener).onPartitionsRevoked(eq(singleton(tp))); assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO)); - verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseApplicationEvent.class), any()); + verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseEvent.class), any()); verify(listener).onPartitionsRevoked(eq(singleton(tp))); assertEquals(emptySet(), subscriptions.assignedPartitions()); } @@ -659,7 +729,7 @@ public void testCompleteQuietly() { assertDoesNotThrow(() -> consumer.completeQuietly(() -> { throw new KafkaException("Test exception"); }, "test", exception)); - assertTrue(exception.get() instanceof KafkaException); + assertInstanceOf(KafkaException.class, exception.get()); } @Test @@ -676,8 +746,8 @@ public void testAutoCommitSyncEnabled() { consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); - consumer.maybeAutoCommitSync(true, time.timer(100), null); - verify(applicationEventHandler).add(any(SyncCommitApplicationEvent.class)); + consumer.maybeAutoCommitSync(true, time.timer(100)); + verify(applicationEventHandler).add(any(SyncCommitEvent.class)); } @Test @@ -694,8 +764,8 @@ public void testAutoCommitSyncDisabled() { consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); - consumer.maybeAutoCommitSync(false, time.timer(100), null); - verify(applicationEventHandler, never()).add(any(SyncCommitApplicationEvent.class)); + consumer.maybeAutoCommitSync(false, time.timer(100)); + verify(applicationEventHandler, never()).add(any(SyncCommitEvent.class)); } private void assertMockCommitCallbackInvoked(final Executable task, @@ -729,7 +799,7 @@ public void testAssign() { consumer.assign(singleton(tp)); assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().contains(tp)); - verify(applicationEventHandler).add(any(AssignmentChangeApplicationEvent.class)); + verify(applicationEventHandler).add(any(AssignmentChangeEvent.class)); verify(applicationEventHandler).add(any(NewTopicsMetadataUpdateRequestEvent.class)); } @@ -781,7 +851,7 @@ public void testBeginningOffsets() { Map expectedOffsets = expectedOffsetsAndTimestamp.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); assertEquals(expectedOffsets, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); } @@ -792,13 +862,13 @@ public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailur Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + "processing List Offsets event"); doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet( - any(ListOffsetsApplicationEvent.class), + any(ListOffsetsEvent.class), any()); Throwable consumerError = assertThrows(KafkaException.class, () -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); assertEquals(eventProcessingFailure, consumerError); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); } @@ -810,7 +880,7 @@ public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { () -> consumer.beginningOffsets( Collections.singletonList(new TopicPartition("t1", 0)), Duration.ofMillis(1))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); } @@ -850,7 +920,7 @@ public void testOffsetsForTimes() { Map result = assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); assertEquals(expectedResult, result); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); } @@ -869,7 +939,7 @@ public void testOffsetsForTimesWithZeroTimeout() { assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ZERO)); assertEquals(expectedResult, result); - verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsApplicationEvent.class), + verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); } @@ -880,21 +950,46 @@ public void testWakeupCommitted() { doAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); Timer timer = invocation.getArgument(1); - assertInstanceOf(FetchCommittedOffsetsApplicationEvent.class, event); + assertInstanceOf(FetchCommittedOffsetsEvent.class, event); assertTrue(event.future().isCompletedExceptionally()); return ConsumerUtils.getResult(event.future(), timer); }) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsEvent.class), any(Timer.class)); consumer.wakeup(); assertThrows(WakeupException.class, () -> consumer.committed(offsets.keySet())); assertNull(consumer.wakeupTrigger().getPendingTask()); } + @Test + public void testNoWakeupInCloseCommit() { + TopicPartition tp = new TopicPartition("topic1", 0); + consumer = newConsumer(); + consumer.assign(Collections.singleton(tp)); + doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); + consumer.seek(tp, 10); + consumer.wakeup(); + + AtomicReference capturedEvent = new AtomicReference<>(); + doAnswer(invocation -> { + ApplicationEvent event = invocation.getArgument(0); + if (event instanceof SyncCommitEvent) { + capturedEvent.set((SyncCommitEvent) event); + } + return null; + }).when(applicationEventHandler).add(any()); + + consumer.close(Duration.ZERO); + + // A commit was triggered and not completed exceptionally by the wakeup + assertNotNull(capturedEvent.get()); + assertFalse(capturedEvent.get().future().isCompletedExceptionally()); + } + @Test public void testInterceptorAutoCommitOnClose() { - Properties props = requiredConsumerPropertiesAndGroupId("test-id"); + Properties props = requiredConsumerConfigAndGroupId("test-id"); props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName()); props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); @@ -910,7 +1005,7 @@ public void testInterceptorAutoCommitOnClose() { @Test public void testInterceptorCommitSync() { - Properties props = requiredConsumerPropertiesAndGroupId("test-id"); + Properties props = requiredConsumerConfigAndGroupId("test-id"); props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName()); props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); @@ -925,7 +1020,7 @@ public void testInterceptorCommitSync() { @Test public void testNoInterceptorCommitSyncFailed() { - Properties props = requiredConsumerPropertiesAndGroupId("test-id"); + Properties props = requiredConsumerConfigAndGroupId("test-id"); props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName()); props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); @@ -941,7 +1036,7 @@ public void testNoInterceptorCommitSyncFailed() { @Test public void testInterceptorCommitAsync() { - Properties props = requiredConsumerPropertiesAndGroupId("test-id"); + Properties props = requiredConsumerConfigAndGroupId("test-id"); props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName()); props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); @@ -958,7 +1053,7 @@ public void testInterceptorCommitAsync() { @Test public void testNoInterceptorCommitAsyncFailed() { - Properties props = requiredConsumerPropertiesAndGroupId("test-id"); + Properties props = requiredConsumerConfigAndGroupId("test-id"); props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName()); props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); @@ -1011,7 +1106,7 @@ public void testSubscribeGeneratesEvent() { consumer.subscribe(singletonList(topic)); assertEquals(singleton(topic), consumer.subscription()); assertTrue(consumer.assignment().isEmpty()); - verify(applicationEventHandler).add(ArgumentMatchers.isA(SubscriptionChangeApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(SubscriptionChangeEvent.class)); } @Test @@ -1023,7 +1118,7 @@ public void testUnsubscribeGeneratesUnsubscribeEvent() { assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().isEmpty()); - verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeEvent.class)); } @Test @@ -1034,7 +1129,7 @@ public void testSubscribeToEmptyListActsAsUnsubscribe() { consumer.subscribe(Collections.emptyList()); assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().isEmpty()); - verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); + verify(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeEvent.class)); } @Test @@ -1058,7 +1153,7 @@ public void testSubscriptionOnEmptyTopic() { @Test public void testGroupMetadataAfterCreationWithGroupIdIsNull() { - final Properties props = requiredConsumerProperties(); + final Properties props = requiredConsumerConfig(); final ConsumerConfig config = new ConsumerConfig(props); consumer = newConsumer(config); @@ -1075,7 +1170,7 @@ public void testGroupMetadataAfterCreationWithGroupIdIsNull() { @Test public void testGroupMetadataAfterCreationWithGroupIdIsNotNull() { final String groupId = "consumerGroupA"; - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); + final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId)); consumer = newConsumer(config); final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); @@ -1090,7 +1185,7 @@ public void testGroupMetadataAfterCreationWithGroupIdIsNotNull() { public void testGroupMetadataAfterCreationWithGroupIdIsNotNullAndGroupInstanceIdSet() { final String groupId = "consumerGroupA"; final String groupInstanceId = "groupInstanceId1"; - final Properties props = requiredConsumerPropertiesAndGroupId(groupId); + final Properties props = requiredConsumerConfigAndGroupId(groupId); props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId); final ConsumerConfig config = new ConsumerConfig(props); consumer = newConsumer(config); @@ -1103,164 +1198,65 @@ public void testGroupMetadataAfterCreationWithGroupIdIsNotNullAndGroupInstanceId assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId()); } - @Test - public void testGroupMetadataUpdateSingleCall() { + private MemberStateListener captureGroupMetadataUpdateListener(final MockedStatic requestManagers) { + ArgumentCaptor applicationThreadMemberStateListener = ArgumentCaptor.forClass(MemberStateListener.class); + requestManagers.verify(() -> RequestManagers.supplier( + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + applicationThreadMemberStateListener.capture() + )); + return applicationThreadMemberStateListener.getValue(); + } + + @Test + public void testGroupMetadataUpdate() { final String groupId = "consumerGroupA"; - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - consumer = newConsumer(config); - - doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); - completeFetchedCommittedOffsetApplicationEventSuccessfully(mkMap()); - - final int generation = 1; - final String memberId = "newMemberId"; - final ConsumerGroupMetadata expectedGroupMetadata = new ConsumerGroupMetadata( - groupId, - generation, - memberId, - Optional.empty() - ); - final GroupMetadataUpdateEvent groupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - generation, - memberId - ); - backgroundEventQueue.add(groupMetadataUpdateEvent); - consumer.assign(singletonList(new TopicPartition("topic", 0))); - consumer.poll(Duration.ZERO); - - final ConsumerGroupMetadata actualGroupMetadata = consumer.groupMetadata(); - - assertEquals(expectedGroupMetadata, actualGroupMetadata); - - final ConsumerGroupMetadata secondActualGroupMetadataWithoutUpdate = consumer.groupMetadata(); - - assertEquals(expectedGroupMetadata, secondActualGroupMetadataWithoutUpdate); - } - - @Test - public void testPollNotReturningRecordsIfGenerationUnknownAndGroupManagementIsUsedWithTopics() { - testPollNotReturningRecordsIfGenerationUnknownAndGroupManagementIsUsed(() -> { - consumer.subscribe(singletonList("topic")); - }); - } - - @Test - public void testPollNotReturningRecordsIfGenerationUnknownAndGroupManagementIsUsedWithPattern() { - testPollNotReturningRecordsIfGenerationUnknownAndGroupManagementIsUsed(() -> { - when(metadata.fetch()).thenReturn(Cluster.empty()); - consumer.subscribe(Pattern.compile("topic")); - }); - } - - private void testPollNotReturningRecordsIfGenerationUnknownAndGroupManagementIsUsed(final Runnable subscription) { - final String groupId = "consumerGroupA"; - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - consumer = newConsumer(config); - subscription.run(); - - consumer.poll(Duration.ZERO); - - verify(fetchCollector, never()).collectFetch(any(FetchBuffer.class)); - } - - @Test - public void testPollReturningRecordsIfGroupIdSetAndGroupManagementIsNotUsed() { - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId("consumerGroupA")); - testPollReturningRecordsIfGroupMetadataHasUnknownGenerationAndGroupManagementIsNotUsed(config); - } - - @Test - public void testPollReturningRecordsIfGroupIdNotSetAndGroupManagementIsNotUsed() { - final ConsumerConfig config = new ConsumerConfig(requiredConsumerProperties()); - testPollReturningRecordsIfGroupMetadataHasUnknownGenerationAndGroupManagementIsNotUsed(config); - } - - private void testPollReturningRecordsIfGroupMetadataHasUnknownGenerationAndGroupManagementIsNotUsed(final ConsumerConfig config) { - final String topic = "topic"; - final TopicPartition topicPartition = new TopicPartition(topic, 0); - consumer = newConsumer(config); - consumer.assign(singletonList(topicPartition)); - final List> records = singletonList( - new ConsumerRecord<>(topic, 0, 2, "key1", "value1") - ); - when(fetchCollector.collectFetch(any(FetchBuffer.class))) - .thenReturn(Fetch.forPartition(topicPartition, records, true)); - completeFetchedCommittedOffsetApplicationEventSuccessfully(mkMap()); - - consumer.poll(Duration.ZERO); - - verify(fetchCollector).collectFetch(any(FetchBuffer.class)); - } - - @Test - public void testPollReturningRecordIfGenerationKnownAndGroupManagementIsUsedWithTopics() { - final String topic = "topic"; - testPollReturningRecordIfGenerationKnownAndGroupManagementIsUsed( - topic, - () -> { - consumer.subscribe(singletonList(topic)); - }); - } - - @Test - public void testPollReturningRecordIfGenerationKnownAndGroupManagementIsUsedWithPattern() { - final String topic = "topic"; - testPollReturningRecordIfGenerationKnownAndGroupManagementIsUsed( - topic, - () -> { - when(metadata.fetch()).thenReturn(Cluster.empty()); - consumer.subscribe(Pattern.compile(topic)); - }); - } - - private void testPollReturningRecordIfGenerationKnownAndGroupManagementIsUsed(final String topic, - final Runnable subscription) { - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId("consumerGroupA")); - final int generation = 1; - final String memberId = "newMemberId"; - final GroupMetadataUpdateEvent groupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - generation, - memberId - ); - backgroundEventQueue.add(groupMetadataUpdateEvent); - final TopicPartition topicPartition = new TopicPartition(topic, 0); - final List> records = singletonList( - new ConsumerRecord<>(topic, 0, 2, "key1", "value1") - ); - when(fetchCollector.collectFetch(any(FetchBuffer.class))) - .thenReturn(Fetch.forPartition(topicPartition, records, true)); - consumer = newConsumer(config); - subscription.run(); - - consumer.poll(Duration.ZERO); - - verify(fetchCollector).collectFetch(any(FetchBuffer.class)); + final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId)); + try (final MockedStatic requestManagers = mockStatic(RequestManagers.class)) { + consumer = newConsumer(config); + final ConsumerGroupMetadata oldGroupMetadata = consumer.groupMetadata(); + final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers); + final int expectedMemberEpoch = 42; + final String expectedMemberId = "memberId"; + groupMetadataUpdateListener.onMemberEpochUpdated( + Optional.of(expectedMemberEpoch), + Optional.of(expectedMemberId) + ); + final ConsumerGroupMetadata newGroupMetadata = consumer.groupMetadata(); + assertEquals(oldGroupMetadata.groupId(), newGroupMetadata.groupId()); + assertEquals(expectedMemberId, newGroupMetadata.memberId()); + assertEquals(expectedMemberEpoch, newGroupMetadata.generationId()); + assertEquals(oldGroupMetadata.groupInstanceId(), newGroupMetadata.groupInstanceId()); + } } @Test public void testGroupMetadataIsResetAfterUnsubscribe() { final String groupId = "consumerGroupA"; - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); - consumer = newConsumer(config); - consumer.subscribe(singletonList("topic")); - final int generation = 1; - final String memberId = "newMemberId"; - final ConsumerGroupMetadata groupMetadataAfterSubscription = new ConsumerGroupMetadata( - groupId, - generation, - memberId, - Optional.empty() - ); - final GroupMetadataUpdateEvent groupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - generation, - memberId - ); - backgroundEventQueue.add(groupMetadataUpdateEvent); - when(fetchCollector.collectFetch(any(FetchBuffer.class))).thenReturn(Fetch.empty()); - consumer.poll(Duration.ZERO); - - assertEquals(groupMetadataAfterSubscription, consumer.groupMetadata()); - + final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId)); + try (final MockedStatic requestManagers = mockStatic(RequestManagers.class)) { + consumer = newConsumer(config); + final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers); + consumer.subscribe(singletonList("topic")); + final int memberEpoch = 42; + final String memberId = "memberId"; + groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), Optional.of(memberId)); + final ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); + assertNotEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, groupMetadata.generationId()); + assertNotEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId()); + } completeUnsubscribeApplicationEventSuccessfully(); consumer.unsubscribe(); @@ -1271,7 +1267,6 @@ public void testGroupMetadataIsResetAfterUnsubscribe() { JoinGroupRequest.UNKNOWN_MEMBER_ID, Optional.empty() ); - assertEquals(groupMetadataAfterUnsubscription, consumer.groupMetadata()); } @@ -1352,12 +1347,12 @@ private static Stream listenerCallbacksInvokeSource() { @Test public void testBackgroundError() { final String groupId = "consumerGroupA"; - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); + final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId)); consumer = newConsumer(config); final KafkaException expectedException = new KafkaException("Nobody expects the Spanish Inquisition"); - final ErrorBackgroundEvent errorBackgroundEvent = new ErrorBackgroundEvent(expectedException); - backgroundEventQueue.add(errorBackgroundEvent); + final ErrorEvent errorEvent = new ErrorEvent(expectedException); + backgroundEventQueue.add(errorEvent); consumer.assign(singletonList(new TopicPartition("topic", 0))); final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); @@ -1367,15 +1362,15 @@ public void testBackgroundError() { @Test public void testMultipleBackgroundErrors() { final String groupId = "consumerGroupA"; - final ConsumerConfig config = new ConsumerConfig(requiredConsumerPropertiesAndGroupId(groupId)); + final ConsumerConfig config = new ConsumerConfig(requiredConsumerConfigAndGroupId(groupId)); consumer = newConsumer(config); final KafkaException expectedException1 = new KafkaException("Nobody expects the Spanish Inquisition"); - final ErrorBackgroundEvent errorBackgroundEvent1 = new ErrorBackgroundEvent(expectedException1); - backgroundEventQueue.add(errorBackgroundEvent1); + final ErrorEvent errorEvent1 = new ErrorEvent(expectedException1); + backgroundEventQueue.add(errorEvent1); final KafkaException expectedException2 = new KafkaException("Spam, Spam, Spam"); - final ErrorBackgroundEvent errorBackgroundEvent2 = new ErrorBackgroundEvent(expectedException2); - backgroundEventQueue.add(errorBackgroundEvent2); + final ErrorEvent errorEvent2 = new ErrorEvent(expectedException2); + backgroundEventQueue.add(errorEvent2); consumer.assign(singletonList(new TopicPartition("topic", 0))); final KafkaException exception = assertThrows(KafkaException.class, () -> consumer.poll(Duration.ZERO)); @@ -1385,7 +1380,7 @@ public void testMultipleBackgroundErrors() { @Test public void testGroupRemoteAssignorUnusedIfGroupIdUndefined() { - final Properties props = requiredConsumerProperties(); + final Properties props = requiredConsumerConfig(); props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); final ConsumerConfig config = new ConsumerConfig(props); consumer = newConsumer(config); @@ -1395,7 +1390,7 @@ public void testGroupRemoteAssignorUnusedIfGroupIdUndefined() { @Test public void testGroupRemoteAssignorUnusedInGenericProtocol() { - final Properties props = requiredConsumerProperties(); + final Properties props = requiredConsumerConfig(); props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name().toLowerCase(Locale.ROOT)); props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); @@ -1407,7 +1402,7 @@ public void testGroupRemoteAssignorUnusedInGenericProtocol() { @Test public void testGroupRemoteAssignorUsedInConsumerProtocol() { - final Properties props = requiredConsumerProperties(); + final Properties props = requiredConsumerConfig(); props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA"); props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT)); props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor"); @@ -1419,7 +1414,7 @@ public void testGroupRemoteAssignorUsedInConsumerProtocol() { @Test public void testGroupIdNull() { - final Properties props = requiredConsumerProperties(); + final Properties props = requiredConsumerConfig(); props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); final ConsumerConfig config = new ConsumerConfig(props); @@ -1431,7 +1426,7 @@ public void testGroupIdNull() { @Test public void testGroupIdNotNullAndValid() { - final Properties props = requiredConsumerPropertiesAndGroupId("consumerGroupA"); + final Properties props = requiredConsumerConfigAndGroupId("consumerGroupA"); props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 10000); props.put(THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, true); final ConsumerConfig config = new ConsumerConfig(props); @@ -1465,18 +1460,17 @@ public void testEnsurePollEventSentOnConsumerPoll() { final TopicPartition tp = new TopicPartition("topic", 0); final List> records = singletonList( new ConsumerRecord<>("topic", 0, 2, "key1", "value1")); - backgroundEventQueue.add(new GroupMetadataUpdateEvent(1, "memberId")); doAnswer(invocation -> Fetch.forPartition(tp, records, true)) .when(fetchCollector) .collectFetch(Mockito.any(FetchBuffer.class)); consumer.subscribe(singletonList("topic1")); consumer.poll(Duration.ofMillis(100)); - verify(applicationEventHandler).add(any(PollApplicationEvent.class)); + verify(applicationEventHandler).add(any(PollEvent.class)); } private void testInvalidGroupId(final String groupId) { - final Properties props = requiredConsumerPropertiesAndGroupId(groupId); + final Properties props = requiredConsumerConfigAndGroupId(groupId); final ConsumerConfig config = new ConsumerConfig(props); final Exception exception = assertThrows( @@ -1487,20 +1481,12 @@ private void testInvalidGroupId(final String groupId) { assertEquals("Failed to construct kafka consumer", exception.getMessage()); } - private Properties requiredConsumerPropertiesAndGroupId(final String groupId) { - final Properties props = requiredConsumerProperties(); + private Properties requiredConsumerConfigAndGroupId(final String groupId) { + final Properties props = requiredConsumerConfig(); props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); return props; } - private Properties requiredConsumerProperties() { - final Properties props = new Properties(); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091"); - return props; - } - private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean committedOffsetsEnabled) { completeFetchedCommittedOffsetApplicationEventExceptionally(new TimeoutException()); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); @@ -1510,20 +1496,20 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean co consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsEvent.class), ArgumentMatchers.isA(Timer.class)); if (committedOffsetsEnabled) { // Verify there was an FetchCommittedOffsets event and no ResetPositions event verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class), ArgumentMatchers.isA(Timer.class)); } else { // Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class), ArgumentMatchers.isA(Timer.class)); } } @@ -1538,11 +1524,11 @@ private void testRefreshCommittedOffsetsSuccess(Set partitions, consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsApplicationEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class), ArgumentMatchers.isA(Timer.class)); } @Test @@ -1561,13 +1547,6 @@ public void testLongPollWaitIsLimited() { new ConsumerRecord<>(topicName, partition, 3, "key2", "value2") ); - final int generation = 1; - final String memberId = "newMemberId"; - final GroupMetadataUpdateEvent groupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - generation, - memberId - ); - backgroundEventQueue.add(groupMetadataUpdateEvent); // On the first iteration, return no data; on the second, return two records doAnswer(invocation -> { // Mock the subscription being assigned as the first fetch is collected @@ -1697,54 +1676,54 @@ private HashMap mockTimestampToSearch() { private void completeCommitAsyncApplicationEventExceptionally(Exception ex) { doAnswer(invocation -> { - AsyncCommitApplicationEvent event = invocation.getArgument(0); + AsyncCommitEvent event = invocation.getArgument(0); event.future().completeExceptionally(ex); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitEvent.class)); } private void completeCommitSyncApplicationEventExceptionally(Exception ex) { doAnswer(invocation -> { - SyncCommitApplicationEvent event = invocation.getArgument(0); + SyncCommitEvent event = invocation.getArgument(0); event.future().completeExceptionally(ex); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(SyncCommitApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(SyncCommitEvent.class)); } private void completeCommitAsyncApplicationEventSuccessfully() { doAnswer(invocation -> { - AsyncCommitApplicationEvent event = invocation.getArgument(0); + AsyncCommitEvent event = invocation.getArgument(0); event.future().complete(null); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(AsyncCommitEvent.class)); } private void completeCommitSyncApplicationEventSuccessfully() { doAnswer(invocation -> { - SyncCommitApplicationEvent event = invocation.getArgument(0); + SyncCommitEvent event = invocation.getArgument(0); event.future().complete(null); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(SyncCommitApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(SyncCommitEvent.class)); } private void completeFetchedCommittedOffsetApplicationEventSuccessfully(final Map committedOffsets) { doReturn(committedOffsets) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsEvent.class), any(Timer.class)); } private void completeFetchedCommittedOffsetApplicationEventExceptionally(Exception ex) { doThrow(ex) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsApplicationEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsEvent.class), any(Timer.class)); } private void completeUnsubscribeApplicationEventSuccessfully() { doAnswer(invocation -> { - UnsubscribeApplicationEvent event = invocation.getArgument(0); + UnsubscribeEvent event = invocation.getArgument(0); event.future().complete(null); return null; - }).when(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeApplicationEvent.class)); + }).when(applicationEventHandler).add(ArgumentMatchers.isA(UnsubscribeEvent.class)); } private void forceCommitCallbackInvocation() { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java index c27494d69a..e5b276adf7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java @@ -75,10 +75,10 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_ID; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_INSTANCE_ID; -import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX; import static org.apache.kafka.test.TestUtils.assertFutureThrows; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -96,7 +96,6 @@ public class CommitRequestManagerTest { private long retryBackoffMs = 100; private long retryBackoffMaxMs = 1000; - private String consumerMetricGroupPrefix = CONSUMER_METRIC_GROUP_PREFIX; private static final String CONSUMER_COORDINATOR_METRICS = "consumer-coordinator-metrics"; private Node mockedNode = new Node(1, "host1", 9092); private SubscriptionState subscriptionState; @@ -913,7 +912,7 @@ public void testAutoCommitSyncBeforeRevocationRetriesOnRetriableAndStaleEpoch(Er long expirationTimeMs = time.milliseconds() + retryBackoffMs * 2; // Send commit request expected to be retried on STALE_MEMBER_EPOCH error while it does not expire - commitRequestManager.maybeAutoCommitSyncNow(expirationTimeMs); + commitRequestManager.maybeAutoCommitSyncBeforeRevocation(expirationTimeMs); int newEpoch = 8; String memberId = "member1"; @@ -924,7 +923,7 @@ public void testAutoCommitSyncBeforeRevocationRetriesOnRetriableAndStaleEpoch(Er completeOffsetCommitRequestWithError(commitRequestManager, error); - if (error.exception() instanceof RetriableException || error == Errors.STALE_MEMBER_EPOCH) { + if ((error.exception() instanceof RetriableException || error == Errors.STALE_MEMBER_EPOCH) && error != Errors.UNKNOWN_TOPIC_OR_PARTITION) { assertEquals(1, commitRequestManager.pendingRequests.unsentOffsetCommits.size(), "Request to be retried should be added to the outbound queue"); @@ -1317,7 +1316,7 @@ private ClientResponse buildOffsetFetchClientResponse( final Errors error, final boolean disconnected) { AbstractRequest abstractRequest = request.requestBuilder().build(); - assertTrue(abstractRequest instanceof OffsetFetchRequest); + assertInstanceOf(OffsetFetchRequest.class, abstractRequest); OffsetFetchRequest offsetFetchRequest = (OffsetFetchRequest) abstractRequest; OffsetFetchResponse response = new OffsetFetchResponse(error, topicPartitionData); 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 ba0d3bacef..954ed1c11e 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 @@ -127,6 +127,7 @@ import static org.apache.kafka.test.TestUtils.toSet; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -642,7 +643,7 @@ public void testManyInFlightAsyncCommitsWithCoordinatorDisconnect() { coordinator.commitOffsetsAsync(offsets, (offsets1, exception) -> { responses.incrementAndGet(); Throwable cause = exception.getCause(); - assertTrue(cause instanceof DisconnectException, + assertInstanceOf(DisconnectException.class, cause, "Unexpected exception cause type: " + (cause == null ? null : cause.getClass())); }); } @@ -689,7 +690,7 @@ public void onSuccess(ClientResponse value, RequestFuture future) {} @Override public void onFailure(RuntimeException e, RequestFuture future) { - assertTrue(e instanceof DisconnectException, "Unexpected exception type: " + e.getClass()); + assertInstanceOf(DisconnectException.class, e, "Unexpected exception type: " + e.getClass()); assertTrue(coordinator.coordinatorUnknown()); asyncCallbackInvoked.set(true); } @@ -952,7 +953,7 @@ public void testCoordinatorDisconnect() { assertTrue(future.isDone()); assertTrue(future.failed()); - assertTrue(future.exception() instanceof DisconnectException); + assertInstanceOf(DisconnectException.class, future.exception()); assertTrue(coordinator.coordinatorUnknown()); } @@ -2362,8 +2363,8 @@ private void testInFlightRequestsFailedAfterCoordinatorMarkedDead(Errors error) coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(coordinator.coordinatorUnknown()); - assertTrue(firstCommitCallback.exception instanceof RetriableCommitFailedException); - assertTrue(secondCommitCallback.exception instanceof RetriableCommitFailedException); + assertInstanceOf(RetriableCommitFailedException.class, firstCommitCallback.exception); + assertInstanceOf(RetriableCommitFailedException.class, secondCommitCallback.exception); assertEquals(coordinator.inFlightAsyncCommits.get(), 0); } @@ -2597,7 +2598,7 @@ public void testCommitOffsetAsyncFailedWithDefaultCallback() { assertEquals(coordinator.inFlightAsyncCommits.get(), 0); coordinator.invokeCompletedOffsetCommitCallbacks(); assertEquals(invokedBeforeTest + 1, mockOffsetCommitCallback.invoked); - assertTrue(mockOffsetCommitCallback.exception instanceof RetriableCommitFailedException); + assertInstanceOf(RetriableCommitFailedException.class, mockOffsetCommitCallback.exception); } @Test @@ -2614,7 +2615,7 @@ public void testCommitOffsetAsyncCoordinatorNotAvailable() { assertTrue(coordinator.coordinatorUnknown()); assertEquals(1, cb.invoked); - assertTrue(cb.exception instanceof RetriableCommitFailedException); + assertInstanceOf(RetriableCommitFailedException.class, cb.exception); } @Test @@ -2631,7 +2632,7 @@ public void testCommitOffsetAsyncNotCoordinator() { assertTrue(coordinator.coordinatorUnknown()); assertEquals(1, cb.invoked); - assertTrue(cb.exception instanceof RetriableCommitFailedException); + assertInstanceOf(RetriableCommitFailedException.class, cb.exception); } @Test @@ -2648,7 +2649,7 @@ public void testCommitOffsetAsyncDisconnected() { assertTrue(coordinator.coordinatorUnknown()); assertEquals(1, cb.invoked); - assertTrue(cb.exception instanceof RetriableCommitFailedException); + assertInstanceOf(RetriableCommitFailedException.class, cb.exception); } @Test @@ -2794,7 +2795,7 @@ public void testCommitOffsetIllegalGenerationWithNewGeneration() { coordinator.setNewState(AbstractCoordinator.MemberState.PREPARING_REBALANCE); assertTrue(consumerClient.poll(future, time.timer(30000))); - assertTrue(future.exception().getClass().isInstance(Errors.REBALANCE_IN_PROGRESS.exception())); + assertInstanceOf(future.exception().getClass(), Errors.REBALANCE_IN_PROGRESS.exception()); // the generation should not be reset assertEquals(newGen, coordinator.generation()); @@ -2842,7 +2843,7 @@ public void testCommitOffsetIllegalGenerationWithResetGeneration() { coordinator.setNewGeneration(AbstractCoordinator.Generation.NO_GENERATION); assertTrue(consumerClient.poll(future, time.timer(30000))); - assertTrue(future.exception().getClass().isInstance(new CommitFailedException())); + assertInstanceOf(future.exception().getClass(), new CommitFailedException()); // the generation should not be reset assertEquals(AbstractCoordinator.Generation.NO_GENERATION, coordinator.generation()); @@ -2872,7 +2873,7 @@ public void testCommitOffsetUnknownMemberWithNewGeneration() { coordinator.setNewState(AbstractCoordinator.MemberState.PREPARING_REBALANCE); assertTrue(consumerClient.poll(future, time.timer(30000))); - assertTrue(future.exception().getClass().isInstance(Errors.REBALANCE_IN_PROGRESS.exception())); + assertInstanceOf(future.exception().getClass(), Errors.REBALANCE_IN_PROGRESS.exception()); // the generation should not be reset assertEquals(newGen, coordinator.generation()); @@ -2897,7 +2898,7 @@ public void testCommitOffsetUnknownMemberWithResetGeneration() { coordinator.setNewGeneration(AbstractCoordinator.Generation.NO_GENERATION); assertTrue(consumerClient.poll(future, time.timer(30000))); - assertTrue(future.exception().getClass().isInstance(new CommitFailedException())); + assertInstanceOf(future.exception().getClass(), new CommitFailedException()); // the generation should be reset assertEquals(AbstractCoordinator.Generation.NO_GENERATION, coordinator.generation()); @@ -2947,7 +2948,7 @@ public void testCommitOffsetFencedInstanceWithRebalancingGeneration() { coordinator.setNewGeneration(newGen); assertTrue(consumerClient.poll(future, time.timer(30000))); - assertTrue(future.exception().getClass().isInstance(Errors.REBALANCE_IN_PROGRESS.exception())); + assertInstanceOf(future.exception().getClass(), Errors.REBALANCE_IN_PROGRESS.exception()); // the generation should not be reset assertEquals(newGen, coordinator.generation()); @@ -2976,7 +2977,7 @@ public void testCommitOffsetFencedInstanceWithNewGeneration() { coordinator.setNewGeneration(newGen); assertTrue(consumerClient.poll(future, time.timer(30000))); - assertTrue(future.exception().getClass().isInstance(new CommitFailedException())); + assertInstanceOf(future.exception().getClass(), new CommitFailedException()); // the generation should not be reset assertEquals(newGen, coordinator.generation()); 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 186675491d..a07ea42caa 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 @@ -49,6 +49,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -94,7 +95,7 @@ public void sendWithinBackoffPeriodAfterAuthenticationFailure() { final RequestFuture future = consumerClient.send(node, heartbeat()); consumerClient.poll(future); assertTrue(future.failed()); - assertTrue(future.exception() instanceof AuthenticationException, "Expected only an authentication error."); + assertInstanceOf(AuthenticationException.class, future.exception(), "Expected only an authentication error."); time.sleep(30); // wait less than the backoff period assertTrue(client.connectionFailed(node)); @@ -102,7 +103,7 @@ public void sendWithinBackoffPeriodAfterAuthenticationFailure() { final RequestFuture future2 = consumerClient.send(node, heartbeat()); consumerClient.poll(future2); assertTrue(future2.failed()); - assertTrue(future2.exception() instanceof AuthenticationException, "Expected only an authentication error."); + assertInstanceOf(AuthenticationException.class, future2.exception(), "Expected only an authentication error."); } @Test @@ -127,7 +128,7 @@ public void testDisconnectWithUnsentRequests() { consumerClient.disconnectAsync(node); consumerClient.pollNoWakeup(); assertTrue(future.failed()); - assertTrue(future.exception() instanceof DisconnectException); + assertInstanceOf(DisconnectException.class, future.exception()); } @Test @@ -139,7 +140,7 @@ public void testDisconnectWithInFlightRequests() { consumerClient.disconnectAsync(node); consumerClient.pollNoWakeup(); assertTrue(future.failed()); - assertTrue(future.exception() instanceof DisconnectException); + assertInstanceOf(DisconnectException.class, future.exception()); } @Test @@ -159,7 +160,7 @@ public void testTimeoutUnsentRequest() { assertFalse(consumerClient.hasPendingRequests()); assertTrue(future.failed()); - assertTrue(future.exception() instanceof TimeoutException); + assertInstanceOf(TimeoutException.class, future.exception()); } @Test @@ -227,7 +228,7 @@ public void testDisconnectWakesUpPoll() throws Exception { consumerClient.disconnectAsync(node); t.join(); assertTrue(future.failed()); - assertTrue(future.exception() instanceof DisconnectException); + assertInstanceOf(DisconnectException.class, future.exception()); } @Test diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java index a491df417d..e4d492fb58 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java @@ -20,16 +20,16 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; -import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.AsyncCommitApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; +import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ListOffsetsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; -import org.apache.kafka.clients.consumer.internals.events.PollApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ResetPositionsApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.SyncCommitApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.TopicMetadataApplicationEvent; -import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.PollEvent; +import org.apache.kafka.clients.consumer.internals.events.ResetPositionsEvent; +import org.apache.kafka.clients.consumer.internals.events.SyncCommitEvent; +import org.apache.kafka.clients.consumer.internals.events.TopicMetadataEvent; +import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsEvent; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.FindCoordinatorRequestData; @@ -41,6 +41,7 @@ import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -137,7 +138,7 @@ public void testStartupAndTearDown() throws InterruptedException { @Test public void testApplicationEvent() { - ApplicationEvent e = new PollApplicationEvent(100); + ApplicationEvent e = new PollEvent(100); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor, times(1)).process(e); @@ -153,36 +154,39 @@ public void testMetadataUpdateEvent() { @Test public void testAsyncCommitEvent() { - ApplicationEvent e = new AsyncCommitApplicationEvent(new HashMap<>()); + ApplicationEvent e = new AsyncCommitEvent(new HashMap<>()); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(AsyncCommitApplicationEvent.class)); + verify(applicationEventProcessor).process(any(AsyncCommitEvent.class)); } @Test public void testSyncCommitEvent() { - ApplicationEvent e = new SyncCommitApplicationEvent(new HashMap<>(), 100L); + Timer timer = time.timer(100); + ApplicationEvent e = new SyncCommitEvent(new HashMap<>(), timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(SyncCommitApplicationEvent.class)); + verify(applicationEventProcessor).process(any(SyncCommitEvent.class)); } @Test public void testListOffsetsEventIsProcessed() { Map timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L); - ApplicationEvent e = new ListOffsetsApplicationEvent(timestamps, true); + Timer timer = time.timer(100); + ApplicationEvent e = new ListOffsetsEvent(timestamps, true, timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(ListOffsetsApplicationEvent.class)); + verify(applicationEventProcessor).process(any(ListOffsetsEvent.class)); assertTrue(applicationEventsQueue.isEmpty()); } @Test public void testResetPositionsEventIsProcessed() { - ResetPositionsApplicationEvent e = new ResetPositionsApplicationEvent(); + Timer timer = time.timer(100); + ResetPositionsEvent e = new ResetPositionsEvent(timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class)); + verify(applicationEventProcessor).process(any(ResetPositionsEvent.class)); assertTrue(applicationEventsQueue.isEmpty()); } @@ -190,19 +194,21 @@ public void testResetPositionsEventIsProcessed() { public void testResetPositionsProcessFailureIsIgnored() { doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded(); - ResetPositionsApplicationEvent event = new ResetPositionsApplicationEvent(); + Timer timer = time.timer(100); + ResetPositionsEvent event = new ResetPositionsEvent(timer); applicationEventsQueue.add(event); assertDoesNotThrow(() -> consumerNetworkThread.runOnce()); - verify(applicationEventProcessor).process(any(ResetPositionsApplicationEvent.class)); + verify(applicationEventProcessor).process(any(ResetPositionsEvent.class)); } @Test public void testValidatePositionsEventIsProcessed() { - ValidatePositionsApplicationEvent e = new ValidatePositionsApplicationEvent(); + Timer timer = time.timer(100); + ValidatePositionsEvent e = new ValidatePositionsEvent(timer); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(ValidatePositionsApplicationEvent.class)); + verify(applicationEventProcessor).process(any(ValidatePositionsEvent.class)); assertTrue(applicationEventsQueue.isEmpty()); } @@ -211,11 +217,11 @@ public void testAssignmentChangeEvent() { HashMap offset = mockTopicPartitionOffset(); final long currentTimeMs = time.milliseconds(); - ApplicationEvent e = new AssignmentChangeApplicationEvent(offset, currentTimeMs); + ApplicationEvent e = new AssignmentChangeEvent(offset, currentTimeMs); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(AssignmentChangeApplicationEvent.class)); + verify(applicationEventProcessor).process(any(AssignmentChangeEvent.class)); verify(networkClient, times(1)).poll(anyLong(), anyLong()); verify(commitRequestManager, times(1)).updateAutoCommitTimer(currentTimeMs); // Assignment change should generate an async commit (not retried). @@ -224,9 +230,10 @@ public void testAssignmentChangeEvent() { @Test void testFetchTopicMetadata() { - applicationEventsQueue.add(new TopicMetadataApplicationEvent("topic", Long.MAX_VALUE)); + Timer timer = time.timer(Long.MAX_VALUE); + applicationEventsQueue.add(new TopicMetadataEvent("topic", timer)); consumerNetworkThread.runOnce(); - verify(applicationEventProcessor).process(any(TopicMetadataApplicationEvent.class)); + verify(applicationEventProcessor).process(any(TopicMetadataEvent.class)); } @Test @@ -283,8 +290,8 @@ void testEnsureEventsAreCompleted() { coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); prepareOffsetCommitRequest(new HashMap<>(), Errors.NONE, false); - CompletableApplicationEvent event1 = spy(new AsyncCommitApplicationEvent(Collections.emptyMap())); - ApplicationEvent event2 = new AsyncCommitApplicationEvent(Collections.emptyMap()); + CompletableApplicationEvent event1 = spy(new AsyncCommitEvent(Collections.emptyMap())); + ApplicationEvent event2 = new AsyncCommitEvent(Collections.emptyMap()); CompletableFuture future = new CompletableFuture<>(); when(event1.future()).thenReturn(future); applicationEventsQueue.add(event1); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index d6ae629506..d62d3d8a35 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -26,6 +26,7 @@ import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.metrics.RebalanceCallbackMetricsManager; +import org.apache.kafka.clients.consumer.internals.metrics.RebalanceMetricsManager; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.requests.MetadataResponse; @@ -196,18 +197,19 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA gi.groupInstanceId, metrics)); MembershipManager mm = spy( - new MembershipManagerImpl( - gi.groupId, - gi.groupInstanceId, - groupRebalanceConfig.rebalanceTimeoutMs, - gi.serverAssignor, - subscriptions, - commit, - metadata, - logContext, - Optional.empty(), - backgroundEventHandler, - time + new MembershipManagerImpl( + gi.groupId, + gi.groupInstanceId, + groupRebalanceConfig.rebalanceTimeoutMs, + gi.serverAssignor, + subscriptions, + commit, + metadata, + logContext, + Optional.empty(), + backgroundEventHandler, + time, + mock(RebalanceMetricsManager.class) ) ); HeartbeatRequestManager.HeartbeatState heartbeatState = spy(new HeartbeatRequestManager.HeartbeatState( diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java index d7ad1b5573..d4496522c0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManagerTest.java @@ -18,7 +18,7 @@ import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.TimeoutException; @@ -37,6 +37,7 @@ import java.util.Optional; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.argThat; @@ -114,10 +115,10 @@ public void testPropagateAndBackoffAfterFatalError() { expectFindCoordinatorRequest(coordinatorManager, Errors.GROUP_AUTHORIZATION_FAILED); verify(backgroundEventHandler).add(argThat(backgroundEvent -> { - if (!(backgroundEvent instanceof ErrorBackgroundEvent)) + if (!(backgroundEvent instanceof ErrorEvent)) return false; - RuntimeException exception = ((ErrorBackgroundEvent) backgroundEvent).error(); + RuntimeException exception = ((ErrorEvent) backgroundEvent).error(); if (!(exception instanceof GroupAuthorizationException)) return false; @@ -202,7 +203,7 @@ private ClientResponse buildResponse( Errors error ) { AbstractRequest abstractRequest = request.requestBuilder().build(); - assertTrue(abstractRequest instanceof FindCoordinatorRequest); + assertInstanceOf(FindCoordinatorRequest.class, abstractRequest); FindCoordinatorRequest findCoordinatorRequest = (FindCoordinatorRequest) abstractRequest; FindCoordinatorResponse findCoordinatorResponse = diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchCollectorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchCollectorTest.java index 0ca4a18a48..094e346e02 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchCollectorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchCollectorTest.java @@ -40,6 +40,8 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -61,11 +63,17 @@ import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** * This tests the {@link FetchCollector} functionality in addition to what {@link FetcherTest} tests during the course * of its tests. */ +@MockitoSettings(strictness = Strictness.STRICT_STUBS) public class FetchCollectorTest { private final static int DEFAULT_RECORD_COUNT = 10; @@ -75,7 +83,7 @@ public class FetchCollectorTest { private final TopicPartition topicAPartition1 = new TopicPartition("topic-a", 1); private final TopicPartition topicAPartition2 = new TopicPartition("topic-a", 2); private final Set allPartitions = partitions(topicAPartition0, topicAPartition1, topicAPartition2); - private LogContext logContext; + private final LogContext logContext = new LogContext(); private SubscriptionState subscriptions; private FetchConfig fetchConfig; @@ -406,6 +414,228 @@ public void testFetchWithOtherErrors(final Errors error) { assertThrows(IllegalStateException.class, () -> fetchCollector.collectFetch(fetchBuffer)); } + @Test + public void testCollectFetchInitializationWithNullPosition() { + final TopicPartition topicPartition0 = new TopicPartition("topic", 0); + final SubscriptionState subscriptions = mock(SubscriptionState.class); + when(subscriptions.hasValidPosition(topicPartition0)).thenReturn(true); + when(subscriptions.positionOrNull(topicPartition0)).thenReturn(null); + final FetchCollector fetchCollector = createFetchCollector(subscriptions); + final Records records = createRecords(); + FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() + .setPartitionIndex(topicPartition0.partition()) + .setHighWatermark(1000) + .setRecords(records); + final CompletedFetch completedFetch = new CompletedFetchBuilder() + .partitionData(partitionData) + .partition(topicPartition0).build(); + final FetchBuffer fetchBuffer = mock(FetchBuffer.class); + when(fetchBuffer.nextInLineFetch()).thenReturn(null); + when(fetchBuffer.peek()).thenReturn(completedFetch).thenReturn(null); + + final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); + + assertTrue(fetch.isEmpty()); + verify(fetchBuffer).setNextInLineFetch(null); + } + + @Test + public void testCollectFetchInitializationWithUpdateHighWatermarkOnNotAssignedPartition() { + final TopicPartition topicPartition0 = new TopicPartition("topic", 0); + final long fetchOffset = 42; + final long highWatermark = 1000; + final SubscriptionState subscriptions = mock(SubscriptionState.class); + when(subscriptions.hasValidPosition(topicPartition0)).thenReturn(true); + when(subscriptions.positionOrNull(topicPartition0)).thenReturn(new SubscriptionState.FetchPosition(fetchOffset)); + when(subscriptions.tryUpdatingHighWatermark(topicPartition0, highWatermark)).thenReturn(false); + final FetchCollector fetchCollector = createFetchCollector(subscriptions); + final Records records = createRecords(); + FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() + .setPartitionIndex(topicPartition0.partition()) + .setHighWatermark(highWatermark) + .setRecords(records); + final CompletedFetch completedFetch = new CompletedFetchBuilder() + .partitionData(partitionData) + .partition(topicPartition0) + .fetchOffset(fetchOffset).build(); + final FetchBuffer fetchBuffer = mock(FetchBuffer.class); + when(fetchBuffer.nextInLineFetch()).thenReturn(null); + when(fetchBuffer.peek()).thenReturn(completedFetch).thenReturn(null); + + final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); + + assertTrue(fetch.isEmpty()); + verify(fetchBuffer).setNextInLineFetch(null); + } + + @Test + public void testCollectFetchInitializationWithUpdateLogStartOffsetOnNotAssignedPartition() { + final TopicPartition topicPartition0 = new TopicPartition("topic", 0); + final long fetchOffset = 42; + final long highWatermark = 1000; + final long logStartOffset = 10; + final SubscriptionState subscriptions = mock(SubscriptionState.class); + when(subscriptions.hasValidPosition(topicPartition0)).thenReturn(true); + when(subscriptions.positionOrNull(topicPartition0)).thenReturn(new SubscriptionState.FetchPosition(fetchOffset)); + when(subscriptions.tryUpdatingHighWatermark(topicPartition0, highWatermark)).thenReturn(true); + when(subscriptions.tryUpdatingLogStartOffset(topicPartition0, logStartOffset)).thenReturn(false); + final FetchCollector fetchCollector = createFetchCollector(subscriptions); + final Records records = createRecords(); + FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() + .setPartitionIndex(topicPartition0.partition()) + .setHighWatermark(highWatermark) + .setRecords(records) + .setLogStartOffset(logStartOffset); + final CompletedFetch completedFetch = new CompletedFetchBuilder() + .partitionData(partitionData) + .partition(topicPartition0) + .fetchOffset(fetchOffset).build(); + final FetchBuffer fetchBuffer = mock(FetchBuffer.class); + when(fetchBuffer.nextInLineFetch()).thenReturn(null); + when(fetchBuffer.peek()).thenReturn(completedFetch).thenReturn(null); + + final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); + + assertTrue(fetch.isEmpty()); + verify(fetchBuffer).setNextInLineFetch(null); + } + + @Test + public void testCollectFetchInitializationWithUpdateLastStableOffsetOnNotAssignedPartition() { + final TopicPartition topicPartition0 = new TopicPartition("topic", 0); + final long fetchOffset = 42; + final long highWatermark = 1000; + final long logStartOffset = 10; + final long lastStableOffset = 900; + final SubscriptionState subscriptions = mock(SubscriptionState.class); + when(subscriptions.hasValidPosition(topicPartition0)).thenReturn(true); + when(subscriptions.positionOrNull(topicPartition0)).thenReturn(new SubscriptionState.FetchPosition(fetchOffset)); + when(subscriptions.tryUpdatingHighWatermark(topicPartition0, highWatermark)).thenReturn(true); + when(subscriptions.tryUpdatingLogStartOffset(topicPartition0, logStartOffset)).thenReturn(true); + when(subscriptions.tryUpdatingLastStableOffset(topicPartition0, lastStableOffset)).thenReturn(false); + final FetchCollector fetchCollector = createFetchCollector(subscriptions); + final Records records = createRecords(); + FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() + .setPartitionIndex(topicPartition0.partition()) + .setHighWatermark(highWatermark) + .setRecords(records) + .setLogStartOffset(logStartOffset) + .setLastStableOffset(lastStableOffset); + final CompletedFetch completedFetch = new CompletedFetchBuilder() + .partitionData(partitionData) + .partition(topicPartition0) + .fetchOffset(fetchOffset).build(); + final FetchBuffer fetchBuffer = mock(FetchBuffer.class); + when(fetchBuffer.nextInLineFetch()).thenReturn(null); + when(fetchBuffer.peek()).thenReturn(completedFetch).thenReturn(null); + + final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); + + assertTrue(fetch.isEmpty()); + verify(fetchBuffer).setNextInLineFetch(null); + } + + @Test + public void testCollectFetchInitializationWithUpdatePreferredReplicaOnNotAssignedPartition() { + final TopicPartition topicPartition0 = new TopicPartition("topic", 0); + final long fetchOffset = 42; + final long highWatermark = 1000; + final long logStartOffset = 10; + final long lastStableOffset = 900; + final int preferredReadReplicaId = 21; + final SubscriptionState subscriptions = mock(SubscriptionState.class); + when(subscriptions.hasValidPosition(topicPartition0)).thenReturn(true); + when(subscriptions.positionOrNull(topicPartition0)).thenReturn(new SubscriptionState.FetchPosition(fetchOffset)); + when(subscriptions.tryUpdatingHighWatermark(topicPartition0, highWatermark)).thenReturn(true); + when(subscriptions.tryUpdatingLogStartOffset(topicPartition0, logStartOffset)).thenReturn(true); + when(subscriptions.tryUpdatingLastStableOffset(topicPartition0, lastStableOffset)).thenReturn(true); + when(subscriptions.tryUpdatingPreferredReadReplica(eq(topicPartition0), eq(preferredReadReplicaId), any())).thenReturn(false); + final FetchCollector fetchCollector = createFetchCollector(subscriptions); + final Records records = createRecords(); + FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() + .setPartitionIndex(topicPartition0.partition()) + .setHighWatermark(highWatermark) + .setRecords(records) + .setLogStartOffset(logStartOffset) + .setLastStableOffset(lastStableOffset) + .setPreferredReadReplica(preferredReadReplicaId); + final CompletedFetch completedFetch = new CompletedFetchBuilder() + .partitionData(partitionData) + .partition(topicPartition0) + .fetchOffset(fetchOffset).build(); + final FetchBuffer fetchBuffer = mock(FetchBuffer.class); + when(fetchBuffer.nextInLineFetch()).thenReturn(null); + when(fetchBuffer.peek()).thenReturn(completedFetch).thenReturn(null); + + final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); + + assertTrue(fetch.isEmpty()); + verify(fetchBuffer).setNextInLineFetch(null); + } + + @Test + public void testCollectFetchInitializationOffsetOutOfRangeErrorWithNullPosition() { + final TopicPartition topicPartition0 = new TopicPartition("topic", 0); + final SubscriptionState subscriptions = mock(SubscriptionState.class); + when(subscriptions.hasValidPosition(topicPartition0)).thenReturn(true); + when(subscriptions.positionOrNull(topicPartition0)).thenReturn(null); + final FetchCollector fetchCollector = createFetchCollector(subscriptions); + FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() + .setPartitionIndex(topicPartition0.partition()) + .setErrorCode(Errors.OFFSET_OUT_OF_RANGE.code()); + final CompletedFetch completedFetch = new CompletedFetchBuilder() + .partitionData(partitionData) + .partition(topicPartition0).build(); + final FetchBuffer fetchBuffer = mock(FetchBuffer.class); + when(fetchBuffer.nextInLineFetch()).thenReturn(null); + when(fetchBuffer.peek()).thenReturn(completedFetch).thenReturn(null); + + final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); + + assertTrue(fetch.isEmpty()); + verify(fetchBuffer).setNextInLineFetch(null); + } + + @Test + public void testCollectFetchInitializationOffsetOutOfRangeErrorWithOffsetReset() { + final TopicPartition topicPartition0 = new TopicPartition("topic", 0); + final long fetchOffset = 42; + final SubscriptionState subscriptions = mock(SubscriptionState.class); + when(subscriptions.hasValidPosition(topicPartition0)).thenReturn(true); + when(subscriptions.positionOrNull(topicPartition0)).thenReturn(new SubscriptionState.FetchPosition(fetchOffset)); + when(subscriptions.hasDefaultOffsetResetPolicy()).thenReturn(true); + final FetchCollector fetchCollector = createFetchCollector(subscriptions); + FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() + .setPartitionIndex(topicPartition0.partition()) + .setErrorCode(Errors.OFFSET_OUT_OF_RANGE.code()); + final CompletedFetch completedFetch = new CompletedFetchBuilder() + .partitionData(partitionData) + .partition(topicPartition0) + .fetchOffset(fetchOffset).build(); + final FetchBuffer fetchBuffer = mock(FetchBuffer.class); + when(fetchBuffer.nextInLineFetch()).thenReturn(null); + when(fetchBuffer.peek()).thenReturn(completedFetch).thenReturn(null); + + final Fetch fetch = fetchCollector.collectFetch(fetchBuffer); + + assertTrue(fetch.isEmpty()); + verify(subscriptions).requestOffsetResetIfPartitionAssigned(topicPartition0); + verify(fetchBuffer).setNextInLineFetch(null); + } + + private FetchCollector createFetchCollector(final SubscriptionState subscriptions) { + final Properties consumerProps = consumerProps(); + return new FetchCollector<>( + logContext, + mock(ConsumerMetadata.class), + subscriptions, + new FetchConfig(new ConsumerConfig(consumerProps)), + new Deserializers<>(new StringDeserializer(), new StringDeserializer()), + mock(FetchMetricsManager.class), + new MockTime() + ); + } + /** * This is a handy utility method for returning a set from a varargs array. */ @@ -418,14 +648,7 @@ private void buildDependencies() { } private void buildDependencies(int maxPollRecords) { - logContext = new LogContext(); - - Properties p = new Properties(); - p.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - p.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - p.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - p.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, String.valueOf(maxPollRecords)); - + Properties p = consumerProperties(maxPollRecords); ConsumerConfig config = new ConsumerConfig(p); deserializers = new Deserializers<>(new StringDeserializer(), new StringDeserializer()); @@ -456,6 +679,20 @@ private void buildDependencies(int maxPollRecords) { completedFetchBuilder = new CompletedFetchBuilder(); } + private Properties consumerProps() { + return consumerProperties(DEFAULT_MAX_POLL_RECORDS); + } + + private Properties consumerProperties(final int maxPollRecords) { + Properties p = new Properties(); + p.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + p.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + p.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + p.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, String.valueOf(maxPollRecords)); + + return p; + } + private void assign(TopicPartition... partitions) { subscriptions.assignFromUser(partitions(partitions)); } @@ -527,6 +764,10 @@ private class CompletedFetchBuilder { private int recordCount = DEFAULT_RECORD_COUNT; + private TopicPartition topicPartition = topicAPartition0; + + private FetchResponseData.PartitionData partitionData = null; + private Errors error = null; private CompletedFetchBuilder fetchOffset(long fetchOffset) { @@ -544,24 +785,29 @@ private CompletedFetchBuilder error(Errors error) { return this; } - private CompletedFetch build() { - Records records; - ByteBuffer allocate = ByteBuffer.allocate(1024); + private CompletedFetchBuilder partitionData(FetchResponseData.PartitionData partitionData) { + this.partitionData = partitionData; + return this; + } - try (MemoryRecordsBuilder builder = MemoryRecords.builder(allocate, - CompressionType.NONE, - TimestampType.CREATE_TIME, - 0)) { - for (int i = 0; i < recordCount; i++) - builder.append(0L, "key".getBytes(), ("value-" + i).getBytes()); + private CompletedFetchBuilder partition(TopicPartition topicPartition) { + this.topicPartition = topicPartition; + return this; + } - records = builder.build(); - } + private CompletedFetch build() { + Records records = createRecords(recordCount); - FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() + if (partitionData == null) { + partitionData = new FetchResponseData.PartitionData() .setPartitionIndex(topicAPartition0.partition()) .setHighWatermark(1000) .setRecords(records); + } + + if (topicPartition != null) { + partitionData.setPartitionIndex(topicPartition.partition()); + } if (error != null) partitionData.setErrorCode(error.code()); @@ -571,11 +817,29 @@ private CompletedFetch build() { logContext, subscriptions, BufferSupplier.create(), - topicAPartition0, + topicPartition, partitionData, metricsAggregator, fetchOffset, ApiKeys.FETCH.latestVersion()); } } + + private Records createRecords() { + return createRecords(DEFAULT_RECORD_COUNT); + } + + private Records createRecords(final int recordCount) { + ByteBuffer allocate = ByteBuffer.allocate(1024); + + try (MemoryRecordsBuilder builder = MemoryRecords.builder(allocate, + CompressionType.NONE, + TimestampType.CREATE_TIME, + 0)) { + for (int i = 0; i < recordCount; i++) + builder.append(0L, "key".getBytes(), ("value-" + i).getBytes()); + + return builder.build(); + } + } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index 5cf5b9e2d9..653394cd4f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -19,20 +19,23 @@ import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager.HeartbeatRequestState; +import org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager.HeartbeatState; +import org.apache.kafka.clients.consumer.internals.MembershipManager.LocalAssignment; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.GroupMetadataUpdateEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; +import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData.Assignment; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest; +import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest.Builder; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse; import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.serialization.StringDeserializer; @@ -54,10 +57,11 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Properties; import java.util.Random; -import java.util.concurrent.BlockingQueue; +import java.util.SortedSet; import java.util.concurrent.TimeUnit; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_GROUP_INSTANCE_ID; @@ -66,7 +70,9 @@ import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_REMOTE_ASSIGNOR; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_RETRY_BACKOFF_MAX_MS; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_RETRY_BACKOFF_MS; +import static org.apache.kafka.common.utils.Utils.mkSortedSet; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -77,15 +83,11 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; + public class HeartbeatRequestManagerTest { - private long retryBackoffMs = DEFAULT_RETRY_BACKOFF_MS; - private int heartbeatIntervalMs = DEFAULT_HEARTBEAT_INTERVAL_MS; - private int maxPollIntervalMs = DEFAULT_MAX_POLL_INTERVAL_MS; - private long retryBackoffMaxMs = DEFAULT_RETRY_BACKOFF_MAX_MS; private static final String DEFAULT_GROUP_ID = "groupId"; private static final String CONSUMER_COORDINATOR_METRICS = "consumer-coordinator-metrics"; @@ -102,7 +104,6 @@ public class HeartbeatRequestManagerTest { private final String memberId = "member-id"; private final int memberEpoch = 1; private BackgroundEventHandler backgroundEventHandler; - private BlockingQueue backgroundEventQueue; private Metrics metrics; @BeforeEach @@ -118,7 +119,6 @@ private void setUp(Optional groupInfo) { heartbeatRequestState = testBuilder.heartbeatRequestState.orElseThrow(IllegalStateException::new); heartbeatState = testBuilder.heartbeatState.orElseThrow(IllegalStateException::new); backgroundEventHandler = testBuilder.backgroundEventHandler; - backgroundEventQueue = testBuilder.backgroundEventQueue; subscriptions = testBuilder.subscriptions; membershipManager = testBuilder.membershipManager.orElseThrow(IllegalStateException::new); metadata = testBuilder.metadata; @@ -177,7 +177,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); - assertTrue(request.requestBuilder() instanceof ConsumerGroupHeartbeatRequest.Builder); + assertInstanceOf(Builder.class, request.requestBuilder()); ConsumerGroupHeartbeatRequest heartbeatRequest = (ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version); @@ -186,7 +186,7 @@ public void testFirstHeartbeatIncludesRequiredInfoToJoinGroupAndGetAssignments(s assertTrue(heartbeatRequest.data().memberId().isEmpty()); assertEquals(0, heartbeatRequest.data().memberEpoch()); - // Should include subscription and group basic info to start getting assignments. + // Should include subscription and group basic info to start getting assignments, as well as rebalanceTimeoutMs assertEquals(Collections.singletonList(topic), heartbeatRequest.data().subscribedTopicNames()); assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, heartbeatRequest.data().rebalanceTimeoutMs()); assertEquals(DEFAULT_GROUP_ID, heartbeatRequest.data().groupId()); @@ -306,118 +306,79 @@ public void testValidateConsumerGroupHeartbeatRequest(final short version) { new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() .setMemberId(memberId) .setMemberEpoch(memberEpoch)); - membershipManager.onHeartbeatResponseReceived(result.data()); + membershipManager.onHeartbeatSuccess(result.data()); // Create a ConsumerHeartbeatRequest and verify the payload NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); - assertTrue(request.requestBuilder() instanceof ConsumerGroupHeartbeatRequest.Builder); + assertInstanceOf(Builder.class, request.requestBuilder()); ConsumerGroupHeartbeatRequest heartbeatRequest = (ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version); assertEquals(DEFAULT_GROUP_ID, heartbeatRequest.data().groupId()); assertEquals(memberId, heartbeatRequest.data().memberId()); assertEquals(memberEpoch, heartbeatRequest.data().memberEpoch()); - assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, heartbeatRequest.data().rebalanceTimeoutMs()); + assertEquals(10000, heartbeatRequest.data().rebalanceTimeoutMs()); assertEquals(subscribedTopics, heartbeatRequest.data().subscribedTopicNames()); assertEquals(DEFAULT_GROUP_INSTANCE_ID, heartbeatRequest.data().instanceId()); assertEquals(DEFAULT_REMOTE_ASSIGNOR, heartbeatRequest.data().serverAssignor()); } - @Test - public void testConsumerGroupMetadataFirstUpdate() { - final GroupMetadataUpdateEvent groupMetadataUpdateEvent = makeFirstGroupMetadataUpdate(memberId, memberEpoch); - - final GroupMetadataUpdateEvent expectedGroupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - memberEpoch, - memberId + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) + public void testValidateConsumerGroupHeartbeatRequestAssignmentSentWhenLocalEpochChanges(final short version) { + CoordinatorRequestManager coordinatorRequestManager = mock(CoordinatorRequestManager.class); + MembershipManager membershipManager = mock(MembershipManager.class); + BackgroundEventHandler backgroundEventHandler = mock(BackgroundEventHandler.class); + SubscriptionState subscriptionState = mock(SubscriptionState.class); + HeartbeatRequestState requestState = mock(HeartbeatRequestState.class); + HeartbeatState heartbeatState = new HeartbeatState(subscriptionState, membershipManager, DEFAULT_MAX_POLL_INTERVAL_MS); + + HeartbeatRequestManager heartbeatRequestManager = createHeartbeatRequestManager( + coordinatorRequestManager, + membershipManager, + heartbeatState, + requestState, + backgroundEventHandler ); - assertEquals(expectedGroupMetadataUpdateEvent, groupMetadataUpdateEvent); - } - - @Test - public void testConsumerGroupMetadataUpdateWithSameUpdate() { - makeFirstGroupMetadataUpdate(memberId, memberEpoch); - - time.sleep(2000); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); - assertEquals(1, result.unsentRequests.size()); - NetworkClientDelegate.UnsentRequest request = result.unsentRequests.get(0); - ClientResponse responseWithSameUpdate = createHeartbeatResponse(request, Errors.NONE); - request.handler().onComplete(responseWithSameUpdate); - assertEquals(0, backgroundEventQueue.size()); - } + when(membershipManager.shouldHeartbeatNow()).thenReturn(true); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); - @Test - public void testConsumerGroupMetadataUpdateWithMemberIdNullButMemberEpochUpdated() { - makeFirstGroupMetadataUpdate(memberId, memberEpoch); + Uuid topicId = Uuid.randomUuid(); + ConsumerGroupHeartbeatRequestData.TopicPartitions expectedTopicPartitions = + new ConsumerGroupHeartbeatRequestData.TopicPartitions(); + Map> testAssignment = Collections.singletonMap( + topicId, mkSortedSet(0) + ); + expectedTopicPartitions.setTopicId(topicId); + expectedTopicPartitions.setPartitions(Collections.singletonList(0)); - time.sleep(2000); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + // First heartbeat, include assignment + when(membershipManager.currentAssignment()).thenReturn(new LocalAssignment(0, testAssignment)); - assertEquals(1, result.unsentRequests.size()); - NetworkClientDelegate.UnsentRequest request = result.unsentRequests.get(0); - final int updatedMemberEpoch = 2; - ClientResponse responseWithMemberEpochUpdate = createHeartbeatResponseWithMemberIdNull( - request, - Errors.NONE, - updatedMemberEpoch - ); - request.handler().onComplete(responseWithMemberEpochUpdate); - assertEquals(1, backgroundEventQueue.size()); - final BackgroundEvent eventWithUpdatedMemberEpoch = backgroundEventQueue.poll(); - assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type()); - final GroupMetadataUpdateEvent groupMetadataUpdateEvent = (GroupMetadataUpdateEvent) eventWithUpdatedMemberEpoch; - final GroupMetadataUpdateEvent expectedGroupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - updatedMemberEpoch, - memberId - ); - assertEquals(expectedGroupMetadataUpdateEvent, groupMetadataUpdateEvent); - } + ConsumerGroupHeartbeatRequest heartbeatRequest1 = getHeartbeatRequest(heartbeatRequestManager, version); + assertEquals(Collections.singletonList(expectedTopicPartitions), heartbeatRequest1.data().topicPartitions()); - @Test - public void testConsumerGroupMetadataUpdateWithMemberIdUpdatedAndMemberEpochSame() { - makeFirstGroupMetadataUpdate(memberId, memberEpoch); + // Assignment did not change, so no assignment should be sent + ConsumerGroupHeartbeatRequest heartbeatRequest2 = getHeartbeatRequest(heartbeatRequestManager, version); + assertNull(heartbeatRequest2.data().topicPartitions()); - time.sleep(2000); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + // Local epoch bumped, so assignment should be sent + when(membershipManager.currentAssignment()).thenReturn(new LocalAssignment(1, testAssignment)); - assertEquals(1, result.unsentRequests.size()); - NetworkClientDelegate.UnsentRequest request = result.unsentRequests.get(0); - final String updatedMemberId = "updatedMemberId"; - ClientResponse responseWithMemberIdUpdate = createHeartbeatResponse( - request, - Errors.NONE, - updatedMemberId, - memberEpoch - ); - request.handler().onComplete(responseWithMemberIdUpdate); - assertEquals(1, backgroundEventQueue.size()); - final BackgroundEvent eventWithUpdatedMemberEpoch = backgroundEventQueue.poll(); - assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, eventWithUpdatedMemberEpoch.type()); - final GroupMetadataUpdateEvent groupMetadataUpdateEvent = (GroupMetadataUpdateEvent) eventWithUpdatedMemberEpoch; - final GroupMetadataUpdateEvent expectedGroupMetadataUpdateEvent = new GroupMetadataUpdateEvent( - memberEpoch, - updatedMemberId - ); - assertEquals(expectedGroupMetadataUpdateEvent, groupMetadataUpdateEvent); + ConsumerGroupHeartbeatRequest heartbeatRequest3 = getHeartbeatRequest(heartbeatRequestManager, version); + assertEquals(Collections.singletonList(expectedTopicPartitions), heartbeatRequest3.data().topicPartitions()); } - private GroupMetadataUpdateEvent makeFirstGroupMetadataUpdate(final String memberId, final int memberEpoch) { - resetWithZeroHeartbeatInterval(Optional.empty()); - mockStableMember(); - when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); - assertEquals(1, result.unsentRequests.size()); - NetworkClientDelegate.UnsentRequest request = result.unsentRequests.get(0); - ClientResponse firstResponse = createHeartbeatResponse(request, Errors.NONE, memberId, memberEpoch); - request.handler().onComplete(firstResponse); - assertEquals(1, backgroundEventQueue.size()); - final BackgroundEvent event = backgroundEventQueue.poll(); - assertEquals(BackgroundEvent.Type.GROUP_METADATA_UPDATE, event.type()); - return (GroupMetadataUpdateEvent) event; + private ConsumerGroupHeartbeatRequest getHeartbeatRequest(HeartbeatRequestManager heartbeatRequestManager, final short version) { + // Create a ConsumerHeartbeatRequest and verify the payload -- no assignment should be sent + NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(1, pollResult.unsentRequests.size()); + NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); + assertInstanceOf(Builder.class, request.requestBuilder()); + return (ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version); } @ParameterizedTest @@ -440,8 +401,7 @@ public void testHeartbeatResponseOnErrorHandling(final Errors error, final boole switch (error) { case NONE: - verify(backgroundEventHandler).add(any(GroupMetadataUpdateEvent.class)); - verify(membershipManager, times(2)).onHeartbeatResponseReceived(mockResponse.data()); + verify(membershipManager).onHeartbeatSuccess(mockResponse.data()); assertEquals(DEFAULT_HEARTBEAT_INTERVAL_MS, heartbeatRequestState.nextHeartbeatMs(time.milliseconds())); break; @@ -511,7 +471,7 @@ public void testHeartbeatState() { assertEquals(-1, data.rebalanceTimeoutMs()); assertNull(data.subscribedTopicNames()); assertNull(data.serverAssignor()); - assertNull(data.topicPartitions()); + assertEquals(data.topicPartitions(), Collections.emptyList()); membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); @@ -525,10 +485,23 @@ public void testHeartbeatState() { assertEquals(memberId, data.memberId()); assertEquals(0, data.memberEpoch()); assertNull(data.instanceId()); - assertEquals(-1, data.rebalanceTimeoutMs()); + assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, data.rebalanceTimeoutMs()); assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); - assertNull(data.serverAssignor()); - assertNull(data.topicPartitions()); + assertEquals(ConsumerTestBuilder.DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); + assertEquals(Collections.emptyList(), data.topicPartitions()); + membershipManager.onHeartbeatRequestSent(); + assertEquals(MemberState.JOINING, membershipManager.state()); + + membershipManager.transitionToFenced(); + data = heartbeatState.buildRequestData(); + assertEquals(ConsumerTestBuilder.DEFAULT_GROUP_ID, data.groupId()); + assertEquals(memberId, data.memberId()); + assertEquals(0, data.memberEpoch()); + assertNull(data.instanceId()); + assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, data.rebalanceTimeoutMs()); + assertEquals(Collections.singletonList(topic), data.subscribedTopicNames()); + assertEquals(ConsumerTestBuilder.DEFAULT_REMOTE_ASSIGNOR, data.serverAssignor()); + assertEquals(Collections.emptyList(), data.topicPartitions()); membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.JOINING, membershipManager.state()); @@ -547,7 +520,7 @@ public void testHeartbeatState() { .setMemberEpoch(1) .setAssignment(assignmentTopic1)); when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, "topic1")); - membershipManager.onHeartbeatResponseReceived(rs1.data()); + membershipManager.onHeartbeatSuccess(rs1.data()); // We remain in RECONCILING state, as the assignment will be reconciled on the next poll assertEquals(MemberState.RECONCILING, membershipManager.state()); @@ -561,9 +534,9 @@ public void testPollTimerExpiration() { heartbeatRequestState = spy(new HeartbeatRequestManager.HeartbeatRequestState( new LogContext(), time, - heartbeatIntervalMs, - retryBackoffMs, - retryBackoffMaxMs, + DEFAULT_HEARTBEAT_INTERVAL_MS, + DEFAULT_RETRY_BACKOFF_MS, + DEFAULT_RETRY_BACKOFF_MAX_MS, 0)); backgroundEventHandler = mock(BackgroundEventHandler.class); @@ -578,8 +551,8 @@ public void testPollTimerExpiration() { // On poll timer expiration, the member should send a last heartbeat to leave the group // and notify the membership manager - time.sleep(maxPollIntervalMs); - assertHeartbeat(heartbeatRequestManager, heartbeatIntervalMs); + time.sleep(DEFAULT_MAX_POLL_INTERVAL_MS); + assertHeartbeat(heartbeatRequestManager, DEFAULT_HEARTBEAT_INTERVAL_MS); verify(membershipManager).transitionToSendingLeaveGroup(true); verify(heartbeatState).reset(); verify(heartbeatRequestState).reset(); @@ -592,7 +565,7 @@ public void testPollTimerExpiration() { assertTrue(pollTimer.notExpired()); verify(membershipManager).maybeRejoinStaleMember(); when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); - assertHeartbeat(heartbeatRequestManager, heartbeatIntervalMs); + assertHeartbeat(heartbeatRequestManager, DEFAULT_HEARTBEAT_INTERVAL_MS); } /** @@ -607,7 +580,7 @@ public void testPollTimerExpirationShouldNotMarkMemberStaleIfMemberAlreadyLeavin when(membershipManager.shouldSkipHeartbeat()).thenReturn(false); when(membershipManager.isLeavingGroup()).thenReturn(true); - time.sleep(maxPollIntervalMs); + time.sleep(DEFAULT_MAX_POLL_INTERVAL_MS); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); // No transition to leave due to stale member should be triggered, because the member is @@ -630,8 +603,8 @@ public void testHeartbeatMetrics() { new LogContext(), time, 0, // This initial interval should be 0 to ensure heartbeat on the clock - retryBackoffMs, - retryBackoffMaxMs, + DEFAULT_RETRY_BACKOFF_MS, + DEFAULT_RETRY_BACKOFF_MAX_MS, 0); backgroundEventHandler = mock(BackgroundEventHandler.class); heartbeatRequestManager = createHeartbeatRequestManager( @@ -650,11 +623,11 @@ public void testHeartbeatMetrics() { // test poll assertHeartbeat(heartbeatRequestManager, 0); - time.sleep(heartbeatIntervalMs); + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); assertEquals(1.0, getMetric("heartbeat-total").metricValue()); - assertEquals((double) TimeUnit.MILLISECONDS.toSeconds(heartbeatIntervalMs), getMetric("last-heartbeat-seconds-ago").metricValue()); + assertEquals((double) TimeUnit.MILLISECONDS.toSeconds(DEFAULT_HEARTBEAT_INTERVAL_MS), getMetric("last-heartbeat-seconds-ago").metricValue()); - assertHeartbeat(heartbeatRequestManager, heartbeatIntervalMs); + assertHeartbeat(heartbeatRequestManager, DEFAULT_HEARTBEAT_INTERVAL_MS); assertEquals(0.06d, (double) getMetric("heartbeat-rate").metricValue(), 0.005d); assertEquals(2.0, getMetric("heartbeat-total").metricValue()); @@ -708,11 +681,17 @@ private void assertNoHeartbeat(HeartbeatRequestManager hrm) { private void mockStableMember() { membershipManager.onSubscriptionUpdated(); // Heartbeat response without assignment to set the state to STABLE. + when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); + when(subscriptions.rebalanceListener()).thenReturn(Optional.empty()); ConsumerGroupHeartbeatResponse rs1 = new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() .setHeartbeatIntervalMs(DEFAULT_HEARTBEAT_INTERVAL_MS) .setMemberId(memberId) - .setMemberEpoch(memberEpoch)); - membershipManager.onHeartbeatResponseReceived(rs1.data()); + .setMemberEpoch(memberEpoch) + .setAssignment(new Assignment()) + ); + membershipManager.onHeartbeatSuccess(rs1.data()); + membershipManager.poll(time.milliseconds()); + membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); } @@ -749,23 +728,6 @@ private static Collection errorProvider() { private ClientResponse createHeartbeatResponse( final NetworkClientDelegate.UnsentRequest request, final Errors error - ) { - return createHeartbeatResponse(request, error, memberId, memberEpoch); - } - - private ClientResponse createHeartbeatResponseWithMemberIdNull( - final NetworkClientDelegate.UnsentRequest request, - final Errors error, - final int memberEpoch - ) { - return createHeartbeatResponse(request, error, null, memberEpoch); - } - - private ClientResponse createHeartbeatResponse( - final NetworkClientDelegate.UnsentRequest request, - final Errors error, - final String memberId, - final int memberEpoch ) { ConsumerGroupHeartbeatResponseData data = new ConsumerGroupHeartbeatResponseData() .setErrorCode(error.code()) @@ -794,10 +756,10 @@ private ConsumerConfig config() { prop.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); prop.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - prop.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, String.valueOf(maxPollIntervalMs)); - prop.setProperty(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, String.valueOf(retryBackoffMs)); - prop.setProperty(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG, String.valueOf(retryBackoffMaxMs)); - prop.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, String.valueOf(heartbeatIntervalMs)); + prop.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, String.valueOf(DEFAULT_MAX_POLL_INTERVAL_MS)); + prop.setProperty(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, String.valueOf(DEFAULT_RETRY_BACKOFF_MS)); + prop.setProperty(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG, String.valueOf(DEFAULT_RETRY_BACKOFF_MAX_MS)); + prop.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, String.valueOf(DEFAULT_HEARTBEAT_INTERVAL_MS)); return new ConsumerConfig(prop); } @@ -812,7 +774,7 @@ private HeartbeatRequestManager createHeartbeatRequestManager( final HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState, final BackgroundEventHandler backgroundEventHandler) { LogContext logContext = new LogContext(); - pollTimer = time.timer(maxPollIntervalMs); + pollTimer = time.timer(DEFAULT_MAX_POLL_INTERVAL_MS); return new HeartbeatRequestManager( logContext, pollTimer, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java index 3294068b07..cef608a902 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java @@ -16,16 +16,21 @@ */ package org.apache.kafka.clients.consumer.internals; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackCompletedEvent; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.metrics.RebalanceCallbackMetricsManager; +import org.apache.kafka.clients.consumer.internals.metrics.RebalanceMetricsManager; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; +import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData.Assignment; +import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData.TopicPartitions; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest; @@ -102,6 +107,8 @@ public class MembershipManagerImplTest { private BlockingQueue backgroundEventQueue; private BackgroundEventHandler backgroundEventHandler; private Time time; + private Metrics metrics; + private RebalanceMetricsManager rebalanceMetricsManager; @BeforeEach public void setup() { @@ -111,7 +118,9 @@ public void setup() { commitRequestManager = testBuilder.commitRequestManager.orElseThrow(IllegalStateException::new); backgroundEventQueue = testBuilder.backgroundEventQueue; backgroundEventHandler = testBuilder.backgroundEventHandler; - time = testBuilder.time; + time = new MockTime(0); + metrics = new Metrics(time); + rebalanceMetricsManager = new RebalanceMetricsManager(metrics); } @AfterEach @@ -135,15 +144,16 @@ private MembershipManagerImpl createMembershipManager(String groupInstanceId) { return spy(new MembershipManagerImpl( GROUP_ID, Optional.ofNullable(groupInstanceId), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time)); + backgroundEventHandler, time, rebalanceMetricsManager)); } private MembershipManagerImpl createMembershipManagerJoiningGroup(String groupInstanceId, String serverAssignor) { - MembershipManagerImpl manager = new MembershipManagerImpl( + MembershipManagerImpl manager = spy(new MembershipManagerImpl( GROUP_ID, Optional.ofNullable(groupInstanceId), REBALANCE_TIMEOUT, Optional.ofNullable(serverAssignor), subscriptionState, commitRequestManager, - metadata, logContext, Optional.empty(), backgroundEventHandler, time); + metadata, logContext, Optional.empty(), backgroundEventHandler, time, + rebalanceMetricsManager)); manager.transitionToJoining(); return manager; } @@ -160,7 +170,6 @@ public void testMembershipManagerServerAssignor() { @Test public void testMembershipManagerInitSupportsEmptyGroupInstanceId() { createMembershipManagerJoiningGroup(); - createMembershipManagerJoiningGroup(null, null); } @Test @@ -169,7 +178,7 @@ public void testMembershipManagerRegistersForClusterMetadataUpdatesOnFirstJoin() MembershipManagerImpl manager = new MembershipManagerImpl( GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time); + backgroundEventHandler, time, rebalanceMetricsManager); manager.transitionToJoining(); clearInvocations(metadata); @@ -194,27 +203,24 @@ public void testReconcilingWhenReceivingAssignmentFoundInMetadata() { } @Test - public void testTransitionToReconcilingOnlyIfAssignmentReceived() { + public void testTransitionToReconcilingIfEmptyAssignmentReceived() { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); assertEquals(MemberState.JOINING, membershipManager.state()); ConsumerGroupHeartbeatResponse responseWithoutAssignment = - createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(responseWithoutAssignment.data()); - assertNotEquals(MemberState.RECONCILING, membershipManager.state()); + createConsumerGroupHeartbeatResponse(new Assignment()); + membershipManager.onHeartbeatSuccess(responseWithoutAssignment.data()); + assertEquals(MemberState.RECONCILING, membershipManager.state()); ConsumerGroupHeartbeatResponse responseWithAssignment = createConsumerGroupHeartbeatResponse(createAssignment(true)); - membershipManager.onHeartbeatResponseReceived(responseWithAssignment.data()); + membershipManager.onHeartbeatSuccess(responseWithAssignment.data()); assertEquals(MemberState.RECONCILING, membershipManager.state()); } @Test public void testMemberIdAndEpochResetOnFencedMembers() { - MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); - ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); - assertEquals(MemberState.STABLE, membershipManager.state()); + MembershipManagerImpl membershipManager = createMemberInStableState(); assertEquals(MEMBER_ID, membershipManager.memberId()); assertEquals(MEMBER_EPOCH, membershipManager.memberEpoch()); @@ -227,11 +233,7 @@ public void testMemberIdAndEpochResetOnFencedMembers() { @Test public void testTransitionToFatal() { - MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); - ConsumerGroupHeartbeatResponse heartbeatResponse = - createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); - assertEquals(MemberState.STABLE, membershipManager.state()); + MembershipManagerImpl membershipManager = createMemberInStableState(null); assertEquals(MEMBER_ID, membershipManager.memberId()); assertEquals(MEMBER_EPOCH, membershipManager.memberEpoch()); @@ -246,7 +248,7 @@ public void testTransitionToFailedWhenTryingToJoin() { MembershipManagerImpl membershipManager = new MembershipManagerImpl( GROUP_ID, Optional.empty(), REBALANCE_TIMEOUT, Optional.empty(), subscriptionState, commitRequestManager, metadata, logContext, Optional.empty(), - backgroundEventHandler, time); + backgroundEventHandler, time, rebalanceMetricsManager); assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); membershipManager.transitionToJoining(); @@ -264,6 +266,7 @@ public void testFencingWhenStateIsStable() { @Test public void testListenersGetNotifiedOnTransitionsToFatal() { + when(subscriptionState.rebalanceListener()).thenReturn(Optional.empty()); MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); MemberStateListener listener = mock(MemberStateListener.class); membershipManager.registerStateListener(listener); @@ -279,6 +282,7 @@ public void testListenersGetNotifiedOnTransitionsToFatal() { @Test public void testListenersGetNotifiedOnTransitionsToLeavingGroup() { + when(subscriptionState.rebalanceListener()).thenReturn(Optional.empty()); MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); MemberStateListener listener = mock(MemberStateListener.class); membershipManager.registerStateListener(listener); @@ -299,7 +303,7 @@ public void testListenersGetNotifiedOfMemberEpochUpdatesOnlyIfItChanges() { membershipManager.registerStateListener(listener); int epoch = 5; - membershipManager.onHeartbeatResponseReceived(new ConsumerGroupHeartbeatResponseData() + membershipManager.onHeartbeatSuccess(new ConsumerGroupHeartbeatResponseData() .setErrorCode(Errors.NONE.code()) .setMemberId(MEMBER_ID) .setMemberEpoch(epoch)); @@ -307,7 +311,7 @@ public void testListenersGetNotifiedOfMemberEpochUpdatesOnlyIfItChanges() { verify(listener).onMemberEpochUpdated(Optional.of(epoch), Optional.of(MEMBER_ID)); clearInvocations(listener); - membershipManager.onHeartbeatResponseReceived(new ConsumerGroupHeartbeatResponseData() + membershipManager.onHeartbeatSuccess(new ConsumerGroupHeartbeatResponseData() .setErrorCode(Errors.NONE.code()) .setMemberId(MEMBER_ID) .setMemberEpoch(epoch)); @@ -315,8 +319,11 @@ public void testListenersGetNotifiedOfMemberEpochUpdatesOnlyIfItChanges() { } private void mockStableMember(MembershipManagerImpl membershipManager) { - ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(new Assignment()); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); + membershipManager.poll(time.milliseconds()); + membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); assertEquals(MEMBER_ID, membershipManager.memberId()); assertEquals(MEMBER_EPOCH, membershipManager.memberEpoch()); @@ -463,13 +470,13 @@ public void testDelayedReconciliationResultDiscardedIfMemberNotInReconcilingStat } /** - * This is the case where a member is stuck reconciling an assignment A (waiting on - * metadata, commit or callbacks), and it rejoins (due to fence or unsubscribe/subscribe). If - * the reconciliation of A completes it should not be applied (it should not update the - * assignment on the member or send ack). + * This is the case where a member is stuck reconciling an assignment A (waiting for commit + * to complete), and it rejoins (due to fence or unsubscribe/subscribe). If the + * reconciliation of A completes it should be interrupted, and it should not update the + * assignment on the member or send ack. */ @Test - public void testDelayedReconciliationResultDiscardedIfMemberRejoins() { + public void testDelayedReconciliationResultDiscardedAfterCommitIfMemberRejoins() { MembershipManagerImpl membershipManager = createMemberInStableState(); Uuid topicId1 = Uuid.randomUuid(); String topic1 = "topic1"; @@ -489,25 +496,191 @@ public void testDelayedReconciliationResultDiscardedIfMemberRejoins() { testFencedMemberReleasesAssignmentAndTransitionsToJoining(membershipManager); clearInvocations(subscriptionState); - // Get new assignment A2 after rejoining. This should not trigger a reconciliation just - // yet because there is another on in progress, but should keep the new assignment ready - // to be reconciled next. - Uuid topicId3 = Uuid.randomUuid(); - mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId3, "topic3", owned); - receiveAssignmentAfterRejoin(topicId3, Collections.singletonList(5), membershipManager); - verifyReconciliationNotTriggered(membershipManager); - Map> assignmentAfterRejoin = topicIdPartitionsMap(topicId3, 5); - assertEquals(assignmentAfterRejoin, membershipManager.topicPartitionsAwaitingReconciliation()); + Map> assignmentAfterRejoin = receiveAssignmentAfterRejoin( + Collections.singletonList(5), membershipManager, owned); // Reconciliation completes when the member has already re-joined the group. Should not - // update the subscription state or send ack. + // proceed with the revocation, update the subscription state or send ack. commitResult.complete(null); - verify(subscriptionState, never()).assignFromSubscribed(anyCollection()); - assertNotEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); + assertInitialReconciliationDiscardedAfterRejoin(membershipManager, assignmentAfterRejoin); + } - // Assignment received after rejoining should be ready to reconcile on the next - // reconciliation loop. - assertEquals(assignmentAfterRejoin, membershipManager.topicPartitionsAwaitingReconciliation()); + /** + * This is the case where a member is stuck reconciling an assignment A (waiting for + * onPartitionsRevoked callback to complete), and it rejoins (due to fence or + * unsubscribe/subscribe). When the reconciliation of A completes it should be interrupted, + * and it should not update the assignment on the member or send ack. + */ + @Test + public void testDelayedReconciliationResultDiscardedAfterPartitionsRevokedCallbackIfMemberRejoins() { + MembershipManagerImpl membershipManager = createMemberInStableState(); + Uuid topicId1 = Uuid.randomUuid(); + String topic1 = "topic1"; + List owned = Collections.singletonList(new TopicIdPartition(topicId1, + new TopicPartition(topic1, 0))); + mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId1, topic1, owned); + + // Reconciliation that does not complete stuck on onPartitionsRevoked callback + ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); + ConsumerRebalanceListenerCallbackCompletedEvent callbackCompletedEvent = + mockNewAssignmentStuckOnPartitionsRevokedCallback(membershipManager, topicId1, topic1, + Arrays.asList(1, 2), owned.get(0).topicPartition(), invoker); + Map> assignment1 = topicIdPartitionsMap(topicId1, 1, 2); + assertEquals(assignment1, membershipManager.topicPartitionsAwaitingReconciliation()); + + // Get fenced and rejoin while still reconciling. Get new assignment to reconcile after rejoining. + testFencedMemberReleasesAssignmentAndTransitionsToJoining(membershipManager); + clearInvocations(subscriptionState); + + Map> assignmentAfterRejoin = receiveAssignmentAfterRejoin( + Collections.singletonList(5), membershipManager, owned); + + // onPartitionsRevoked callback completes when the member has already re-joined the group. + // Should not proceed with the assignment, update the subscription state or send ack. + completeCallback(callbackCompletedEvent, membershipManager); + assertInitialReconciliationDiscardedAfterRejoin(membershipManager, assignmentAfterRejoin); + } + + /** + * This is the case where a member is stuck reconciling an assignment A (waiting for + * onPartitionsAssigned callback to complete), and it rejoins (due to fence or + * unsubscribe/subscribe). If the reconciliation of A completes it should be interrupted, and it + * should not update the assignment on the member or send ack. + */ + @Test + public void testDelayedReconciliationResultDiscardedAfterPartitionsAssignedCallbackIfMemberRejoins() { + MembershipManagerImpl membershipManager = createMemberInStableState(); + Uuid topicId1 = Uuid.randomUuid(); + String topic1 = "topic1"; + + // Reconciliation that does not complete stuck on onPartitionsAssigned callback + ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); + int newPartition = 1; + ConsumerRebalanceListenerCallbackCompletedEvent callbackCompletedEvent = + mockNewAssignmentStuckOnPartitionsAssignedCallback(membershipManager, topicId1, + topic1, newPartition, invoker); + Map> assignment1 = topicIdPartitionsMap(topicId1, newPartition); + assertEquals(assignment1, membershipManager.topicPartitionsAwaitingReconciliation()); + + // Get fenced and rejoin while still reconciling. Get new assignment to reconcile after rejoining. + testFencedMemberReleasesAssignmentAndTransitionsToJoining(membershipManager); + clearInvocations(subscriptionState); + + Map> assignmentAfterRejoin = receiveAssignmentAfterRejoin( + Collections.singletonList(5), membershipManager, Collections.emptyList()); + + // onPartitionsAssigned callback completes when the member has already re-joined the group. + // Should not update the subscription state or send ack. + completeCallback(callbackCompletedEvent, membershipManager); + assertInitialReconciliationDiscardedAfterRejoin(membershipManager, assignmentAfterRejoin); + } + + /** + * This is the case where a member rejoins (due to fence). If + * the member gets the same assignment again, it should still reconcile and ack the assignment. + */ + @Test + public void testSameAssignmentReconciledAgainWhenFenced() { + MembershipManagerImpl membershipManager = createMemberInStableState(); + Uuid topic1 = Uuid.randomUuid(); + final Assignment assignment1 = new ConsumerGroupHeartbeatResponseData.Assignment(); + final Assignment assignment2 = new ConsumerGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(Collections.singletonList( + new TopicPartitions() + .setTopicId(topic1) + .setPartitions(Arrays.asList(0, 1, 2)) + )); + when(metadata.topicNames()).thenReturn(Collections.singletonMap(topic1, "topic1")); + assertEquals(toTopicIdPartitionMap(assignment1), membershipManager.currentAssignment().partitions); + + // Receive assignment, wait on commit + membershipManager.onHeartbeatSuccess(createConsumerGroupHeartbeatResponse(assignment2).data()); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + CompletableFuture commitResult = new CompletableFuture<>(); + when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation(anyLong())).thenReturn(commitResult); + membershipManager.poll(time.milliseconds()); + + // Get fenced, commit completes + membershipManager.transitionToFenced(); + + assertEquals(MemberState.JOINING, membershipManager.state()); + assertTrue(membershipManager.currentAssignment().isNone()); + assertTrue(subscriptionState.assignedPartitions().isEmpty()); + + commitResult.complete(null); + + assertEquals(MemberState.JOINING, membershipManager.state()); + assertTrue(membershipManager.currentAssignment().isNone()); + assertTrue(subscriptionState.assignedPartitions().isEmpty()); + + // We have to reconcile & ack the assignment again + membershipManager.onHeartbeatSuccess(createConsumerGroupHeartbeatResponse(assignment1).data()); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + membershipManager.poll(time.milliseconds()); + assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); + membershipManager.onHeartbeatRequestSent(); + assertEquals(MemberState.STABLE, membershipManager.state()); + assertEquals(toTopicIdPartitionMap(assignment1), membershipManager.currentAssignment().partitions); + } + + /** + * This is the case where we receive a new assignment while reconciling an existing one. The intermediate assignment + * is not applied, and a new assignment containing the same partitions is received and reconciled. In all assignments, + * one topic is not resolvable. + * + * We need to make sure that the last assignment is acked and applied, even though the set of partitions does not change. + * In this case, no rebalance listeners are run. + */ + @Test + public void testSameAssignmentReconciledAgainWithMissingTopic() { + MembershipManagerImpl membershipManager = createMemberInStableState(); + Uuid topic1 = Uuid.randomUuid(); + Uuid topic2 = Uuid.randomUuid(); + final Assignment assignment1 = new ConsumerGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(Arrays.asList( + new TopicPartitions().setTopicId(topic1).setPartitions(Collections.singletonList(0)), + new TopicPartitions().setTopicId(topic2).setPartitions(Collections.singletonList(0)) + )); + final Assignment assignment2 = new ConsumerGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(Arrays.asList( + new TopicPartitions().setTopicId(topic1).setPartitions(Arrays.asList(0, 1)), + new TopicPartitions().setTopicId(topic2).setPartitions(Collections.singletonList(0)) + )); + when(metadata.topicNames()).thenReturn(Collections.singletonMap(topic1, "topic1")); + + // Receive assignment - full reconciliation triggered + // stay in RECONCILING state, since an unresolved topic is assigned + membershipManager.onHeartbeatSuccess(createConsumerGroupHeartbeatResponse(assignment1).data()); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + membershipManager.poll(time.milliseconds()); + verifyReconciliationTriggeredAndCompleted(membershipManager, + Collections.singletonList(new TopicIdPartition(topic1, new TopicPartition("topic1", 0))) + ); + membershipManager.onHeartbeatRequestSent(); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + clearInvocations(membershipManager); + + // Receive extended assignment - assignment received but no reconciliation triggered + membershipManager.onHeartbeatSuccess(createConsumerGroupHeartbeatResponse(assignment2).data()); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + verifyReconciliationNotTriggered(membershipManager); + + // Receive original assignment again - full reconciliation not triggered but assignment is acked again + membershipManager.onHeartbeatSuccess(createConsumerGroupHeartbeatResponse(assignment1).data()); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + membershipManager.poll(time.milliseconds()); + assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); + verifyReconciliationNotTriggered(membershipManager); + assertEquals(Collections.singletonMap(topic1, mkSortedSet(0)), membershipManager.currentAssignment().partitions); + assertEquals(mkSet(topic2), membershipManager.topicsAwaitingReconciliation()); + } + + private Map> toTopicIdPartitionMap(final Assignment assignment) { + Map> result = new HashMap<>(); + for (TopicPartitions topicPartitions : assignment.topicPartitions()) { + result.put(topicPartitions.topicId(), new TreeSet<>(topicPartitions.partitions())); + } + return result; } /** @@ -676,6 +849,12 @@ public void testDelayedMetadataUsedToCompleteAssignment() { receiveAssignment(newAssignment, membershipManager); membershipManager.poll(time.milliseconds()); + // No full reconciliation triggered, but assignment needs to be acknowledged. + assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); + assertTrue(membershipManager.shouldHeartbeatNow()); + + membershipManager.onHeartbeatRequestSent(); + verifyReconciliationNotTriggered(membershipManager); assertEquals(MemberState.RECONCILING, membershipManager.state()); assertEquals(Collections.singleton(topicId2), membershipManager.topicsAwaitingReconciliation()); @@ -709,12 +888,12 @@ public void testIgnoreHeartbeatWhenLeavingGroup() { CompletableFuture leaveResult = membershipManager.leaveGroup(); - membershipManager.onHeartbeatResponseReceived(createConsumerGroupHeartbeatResponse(createAssignment(true)).data()); + membershipManager.onHeartbeatSuccess(createConsumerGroupHeartbeatResponse(createAssignment(true)).data()); assertEquals(MemberState.LEAVING, membershipManager.state()); assertEquals(-1, membershipManager.memberEpoch()); assertEquals(MEMBER_ID, membershipManager.memberId()); - assertTrue(membershipManager.currentAssignment().isEmpty()); + assertTrue(membershipManager.currentAssignment().isNone()); assertFalse(leaveResult.isDone(), "Leave group result should not complete until the " + "heartbeat request to leave is sent out."); } @@ -726,7 +905,7 @@ public void testIgnoreHeartbeatResponseWhenNotInGroup(MemberState state) { when(membershipManager.state()).thenReturn(state); ConsumerGroupHeartbeatResponseData responseData = mock(ConsumerGroupHeartbeatResponseData.class); - membershipManager.onHeartbeatResponseReceived(responseData); + membershipManager.onHeartbeatSuccess(responseData); assertEquals(state, membershipManager.state()); verify(responseData, never()).memberId(); @@ -759,11 +938,26 @@ public void testLeaveGroupWhenMemberOwnsAssignment() { new TopicIdPartition(topicId, new TopicPartition(topicName, 1))); verifyReconciliationTriggeredAndCompleted(membershipManager, assignedPartitions); - assertEquals(1, membershipManager.currentAssignment().size()); + assertEquals(1, membershipManager.currentAssignment().partitions.size()); testLeaveGroupReleasesAssignmentAndResetsEpochToSendLeaveGroup(membershipManager); } + @Test + public void testFencedWhenAssignmentEmpty() { + MembershipManager membershipManager = createMemberInStableState(); + + // Clear the assignment + when(subscriptionState.assignedPartitions()).thenReturn(Collections.emptySet()); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(false); + + membershipManager.transitionToFenced(); + + // Make sure to never call `assignFromSubscribed` again + verify(subscriptionState, never()).assignFromSubscribed(Collections.emptySet()); + } + + @Test public void testLeaveGroupWhenMemberAlreadyLeaving() { MembershipManager membershipManager = createMemberInStableState(); @@ -859,10 +1053,7 @@ public void testFatalFailureWhenStateIsUnjoined() { @Test public void testFatalFailureWhenStateIsStable() { - MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); - ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); - assertEquals(MemberState.STABLE, membershipManager.state()); + MembershipManagerImpl membershipManager = createMemberInStableState(null); testStateUpdateOnFatalFailure(membershipManager); } @@ -930,9 +1121,9 @@ public void testUpdateStateFailsOnResponsesWithErrors() { // Updating state with a heartbeat response containing errors cannot be performed and // should fail. ConsumerGroupHeartbeatResponse unknownMemberResponse = - createConsumerGroupHeartbeatResponseWithError(); + createConsumerGroupHeartbeatResponseWithError(Errors.UNKNOWN_MEMBER_ID); assertThrows(IllegalArgumentException.class, - () -> membershipManager.onHeartbeatResponseReceived(unknownMemberResponse.data())); + () -> membershipManager.onHeartbeatSuccess(unknownMemberResponse.data())); } /** @@ -1005,10 +1196,14 @@ public void testNewEmptyAssignmentReplacesPreviousOneWaitingOnMetadata() { receiveEmptyAssignment(membershipManager); verifyReconciliationNotTriggered(membershipManager); + membershipManager.poll(time.milliseconds()); + verifyReconciliationTriggeredAndCompleted(membershipManager, Collections.emptyList()); + + membershipManager.onHeartbeatRequestSent(); + assertEquals(MemberState.STABLE, membershipManager.state()); - verify(subscriptionState, never()).assignFromSubscribed(any()); } @Test @@ -1027,6 +1222,7 @@ public void testNewAssignmentNotInMetadataReplacesPreviousOneWaitingOnMetadata() verifyReconciliationNotTriggered(membershipManager); membershipManager.poll(time.milliseconds()); + membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.RECONCILING, membershipManager.state()); assertFalse(membershipManager.topicsAwaitingReconciliation().isEmpty()); @@ -1099,7 +1295,7 @@ public void testMemberKeepsUnresolvedAssignmentWaitingForMetadataUntilResolved() // Target assignment received again with the same unresolved topic. Client should keep it // as unresolved. clearInvocations(subscriptionState); - membershipManager.onHeartbeatResponseReceived(createConsumerGroupHeartbeatResponse(assignment).data()); + membershipManager.onHeartbeatSuccess(createConsumerGroupHeartbeatResponse(assignment).data()); assertEquals(MemberState.RECONCILING, membershipManager.state()); assertEquals(Collections.singleton(topic2), membershipManager.topicsAwaitingReconciliation()); verify(subscriptionState, never()).assignFromSubscribed(anyCollection()); @@ -1173,6 +1369,9 @@ public void testReconciliationSkippedWhenSameAssignmentReceived() { verify(subscriptionState, never()).assignFromSubscribed(anyCollection()); assertEquals(MemberState.STABLE, membershipManager.state()); + + assertEquals(1.0d, getMetricValue(metrics, rebalanceMetricsManager.rebalanceTotal)); + assertEquals(0.0d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceTotal)); } @Test @@ -1257,7 +1456,7 @@ public void testReconcileNewPartitionsAssignedAndRevoked() { membershipManager.poll(time.milliseconds()); assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); - assertEquals(topicIdPartitionsMap(topicId, 1, 2), membershipManager.currentAssignment()); + assertEquals(topicIdPartitionsMap(topicId, 1, 2), membershipManager.currentAssignment().partitions); assertFalse(membershipManager.reconciliationInProgress()); verify(subscriptionState).assignFromSubscribed(anyCollection()); @@ -1273,7 +1472,8 @@ public void testMetadataUpdatesReconcilesUnresolvedAssignments() { new ConsumerGroupHeartbeatResponseData.TopicPartitions() .setTopicId(topicId) .setPartitions(Arrays.asList(0, 1)))); - MembershipManagerImpl membershipManager = mockJoinAndReceiveAssignment(false, targetAssignment); + MembershipManagerImpl membershipManager = mockJoinAndReceiveAssignment(true, targetAssignment); + membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.RECONCILING, membershipManager.state()); // Should not trigger reconciliation, and request a metadata update. @@ -1302,7 +1502,8 @@ public void testMetadataUpdatesRequestsAnotherUpdateIfNeeded() { new ConsumerGroupHeartbeatResponseData.TopicPartitions() .setTopicId(topicId) .setPartitions(Arrays.asList(0, 1)))); - MembershipManagerImpl membershipManager = mockJoinAndReceiveAssignment(false, targetAssignment); + MembershipManagerImpl membershipManager = mockJoinAndReceiveAssignment(true, targetAssignment); + membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.RECONCILING, membershipManager.state()); // Should not trigger reconciliation, and request a metadata update. @@ -1341,7 +1542,7 @@ public void testRevokePartitionsUsesTopicNamesLocalCacheWhenMetadataNotAvailable partitions.stream().map(p -> new TopicPartition(topicName, p)).collect(Collectors.toSet()); Map> assignedTopicIdPartitions = Collections.singletonMap(topicId, new TreeSet<>(partitions)); - assertEquals(assignedTopicIdPartitions, membershipManager.currentAssignment()); + assertEquals(assignedTopicIdPartitions, membershipManager.currentAssignment().partitions); assertFalse(membershipManager.reconciliationInProgress()); mockAckSent(membershipManager); @@ -1368,15 +1569,12 @@ public void testRevokePartitionsUsesTopicNamesLocalCacheWhenMetadataNotAvailable @Test public void testOnSubscriptionUpdatedTransitionsToJoiningOnlyIfNotInGroup() { MembershipManagerImpl membershipManager = createMemberInStableState(); - verify(membershipManager).transitionToJoining(); - clearInvocations(membershipManager); membershipManager.onSubscriptionUpdated(); verify(membershipManager, never()).transitionToJoining(); } @Test public void testListenerCallbacksBasic() { - // Step 1: set up mocks MembershipManagerImpl membershipManager = createMemberInStableState(); CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener(); ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); @@ -1419,7 +1617,7 @@ public void testListenerCallbacksBasic() { // Step 4: Send ack and make sure we're done and our listener was called appropriately membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); - assertEquals(topicIdPartitionsMap(topicId, 0, 1), membershipManager.currentAssignment()); + assertEquals(topicIdPartitionsMap(topicId, 0, 1), membershipManager.currentAssignment().partitions); assertEquals(0, listener.revokedCount()); assertEquals(1, listener.assignedCount()); @@ -1488,7 +1686,7 @@ public void testListenerCallbacksThrowsErrorOnPartitionsRevoked() { membershipManager.poll(time.milliseconds()); assertEquals(MemberState.RECONCILING, membershipManager.state()); - assertEquals(topicIdPartitionsMap(topicId, 0), membershipManager.currentAssignment()); + assertEquals(topicIdPartitionsMap(topicId, 0), membershipManager.currentAssignment().partitions); assertTrue(membershipManager.reconciliationInProgress()); assertEquals(0, listener.revokedCount()); assertEquals(0, listener.assignedCount()); @@ -1541,7 +1739,7 @@ public void testListenerCallbacksThrowsErrorOnPartitionsAssigned() { membershipManager.poll(time.milliseconds()); assertEquals(MemberState.RECONCILING, membershipManager.state()); - assertEquals(topicIdPartitionsMap(topicId, 0), membershipManager.currentAssignment()); + assertEquals(topicIdPartitionsMap(topicId, 0), membershipManager.currentAssignment().partitions); assertTrue(membershipManager.reconciliationInProgress()); assertEquals(0, listener.revokedCount()); assertEquals(0, listener.assignedCount()); @@ -1642,19 +1840,11 @@ public void testAddedPartitionsNotEnabledAfterFailedOnPartitionsAssignedCallback @Test public void testOnPartitionsLostNoError() { - MembershipManagerImpl membershipManager = createMemberInStableState(); - String topicName = "topic1"; - Uuid topicId = Uuid.randomUuid(); - mockOwnedPartition(membershipManager, topicId, topicName); testOnPartitionsLost(Optional.empty()); } @Test public void testOnPartitionsLostError() { - MembershipManagerImpl membershipManager = createMemberInStableState(); - String topicName = "topic1"; - Uuid topicId = Uuid.randomUuid(); - mockOwnedPartition(membershipManager, topicId, topicName); testOnPartitionsLost(Optional.of(new KafkaException("Intentional error for test"))); } @@ -1669,6 +1859,7 @@ private void assertLeaveGroupDueToExpiredPollAndTransitionToStale(MembershipMana public void testTransitionToLeavingWhileReconcilingDueToStaleMember() { MembershipManagerImpl membershipManager = memberJoinWithAssignment(); clearInvocations(subscriptionState); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); assertEquals(MemberState.RECONCILING, membershipManager.state()); assertLeaveGroupDueToExpiredPollAndTransitionToStale(membershipManager); } @@ -1677,17 +1868,14 @@ public void testTransitionToLeavingWhileReconcilingDueToStaleMember() { public void testTransitionToLeavingWhileJoiningDueToStaleMember() { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); doNothing().when(subscriptionState).assignFromSubscribed(any()); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); assertEquals(MemberState.JOINING, membershipManager.state()); assertLeaveGroupDueToExpiredPollAndTransitionToStale(membershipManager); } @Test public void testTransitionToLeavingWhileStableDueToStaleMember() { - MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); - ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); - doNothing().when(subscriptionState).assignFromSubscribed(any()); - assertEquals(MemberState.STABLE, membershipManager.state()); + MembershipManagerImpl membershipManager = createMemberInStableState(null); assertLeaveGroupDueToExpiredPollAndTransitionToStale(membershipManager); } @@ -1696,6 +1884,7 @@ public void testTransitionToLeavingWhileAcknowledgingDueToStaleMember() { MembershipManagerImpl membershipManager = mockJoinAndReceiveAssignment(true); doNothing().when(subscriptionState).assignFromSubscribed(any()); clearInvocations(subscriptionState); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); assertLeaveGroupDueToExpiredPollAndTransitionToStale(membershipManager); } @@ -1766,6 +1955,7 @@ public void testStaleMemberWaitsForCallbackToRejoinWhenTimerReset() { private MembershipManagerImpl mockStaleMember() { MembershipManagerImpl membershipManager = createMemberInStableState(); doNothing().when(subscriptionState).assignFromSubscribed(any()); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); membershipManager.transitionToSendingLeaveGroup(true); membershipManager.onHeartbeatRequestSent(); return membershipManager; @@ -1778,7 +1968,7 @@ private void mockPartitionOwnedAndNewPartitionAdded(String topicName, Uuid topicId = Uuid.randomUuid(); TopicPartition owned = new TopicPartition(topicName, partitionOwned); when(subscriptionState.assignedPartitions()).thenReturn(Collections.singleton(owned)); - membershipManager.updateCurrentAssignment(Collections.singleton(new TopicIdPartition(topicId, owned))); + membershipManager.updateAssignment(Collections.singletonMap(topicId, mkSortedSet(partitionOwned))); when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); when(subscriptionState.rebalanceListener()).thenReturn(Optional.ofNullable(listener)); @@ -1790,6 +1980,9 @@ private void mockPartitionOwnedAndNewPartitionAdded(String topicName, private void testOnPartitionsLost(Optional lostError) { // Step 1: set up mocks MembershipManagerImpl membershipManager = createMemberInStableState(); + String topicName = "topic1"; + Uuid topicId = Uuid.randomUuid(); + mockOwnedPartition(membershipManager, topicId, topicName); CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener( Optional.empty(), Optional.empty(), @@ -1803,7 +1996,6 @@ private void testOnPartitionsLost(Optional lostError) { // Step 2: put the state machine into the appropriate... state membershipManager.transitionToFenced(); assertEquals(MemberState.FENCED, membershipManager.state()); - assertEquals(Collections.emptyMap(), membershipManager.currentAssignment()); assertEquals(0, listener.revokedCount()); assertEquals(0, listener.assignedCount()); assertEquals(0, listener.lostCount()); @@ -1819,6 +2011,8 @@ private void testOnPartitionsLost(Optional lostError) { true ); + assertTrue(membershipManager.currentAssignment().isNone()); + // Step 4: Receive ack and make sure we're done and our listener was called appropriately membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.JOINING, membershipManager.state()); @@ -1832,8 +2026,8 @@ private ConsumerRebalanceListenerInvoker consumerRebalanceListenerInvoker() { return new ConsumerRebalanceListenerInvoker( new LogContext(), subscriptionState, - new MockTime(1), - new RebalanceCallbackMetricsManager(new Metrics()) + time, + new RebalanceCallbackMetricsManager(new Metrics(time)) ); } @@ -1881,10 +2075,10 @@ private Map> topicIdPartitionsMap(Uuid topicId, int... } private ConsumerRebalanceListenerCallbackCompletedEvent performCallback(MembershipManagerImpl membershipManager, - ConsumerRebalanceListenerInvoker invoker, - ConsumerRebalanceListenerMethodName expectedMethodName, - SortedSet expectedPartitions, - boolean complete) { + ConsumerRebalanceListenerInvoker invoker, + ConsumerRebalanceListenerMethodName expectedMethodName, + SortedSet expectedPartitions, + boolean complete) { // We expect only our enqueued event in the background queue. assertEquals(1, backgroundEventQueue.size()); assertNotNull(backgroundEventQueue.peek()); @@ -1923,23 +2117,194 @@ private void assertStaleMemberLeavesGroupAndClearsAssignment(MembershipManagerIm // Should reset epoch to leave the group and release the assignment (right away because // there is no onPartitionsLost callback defined) verify(subscriptionState).assignFromSubscribed(Collections.emptySet()); - assertTrue(membershipManager.currentAssignment().isEmpty()); + assertTrue(membershipManager.currentAssignment().isNone()); assertTrue(membershipManager.topicsAwaitingReconciliation().isEmpty()); assertEquals(LEAVE_GROUP_MEMBER_EPOCH, membershipManager.memberEpoch()); } @Test public void testMemberJoiningTransitionsToStableWhenReceivingEmptyAssignment() { - MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(null); + MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); + when(subscriptionState.rebalanceListener()).thenReturn(Optional.empty()); assertEquals(MemberState.JOINING, membershipManager.state()); receiveEmptyAssignment(membershipManager); verifyReconciliationNotTriggered(membershipManager); membershipManager.poll(time.milliseconds()); + membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); } + @Test + public void testMemberJoiningCallsRebalanceListenerWhenReceivingEmptyAssignment() { + CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener(); + ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); + + when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); + MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); + receiveEmptyAssignment(membershipManager); + + assertEquals(MemberState.RECONCILING, membershipManager.state()); + assertEquals(0, listener.assignedCount()); + assertEquals(0, listener.revokedCount()); + assertEquals(0, listener.lostCount()); + + verifyReconciliationNotTriggered(membershipManager); + membershipManager.poll(time.milliseconds()); + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + new TreeSet<>(Collections.emptyList()), + true + ); + + assertEquals(1, listener.assignedCount()); + assertEquals(0, listener.revokedCount()); + assertEquals(0, listener.lostCount()); + } + + @Test + public void testMetricsWhenHeartbeatFailed() { + MembershipManagerImpl membershipManager = createMemberInStableState(); + membershipManager.onHeartbeatFailure(); + + // Not expecting rebalance failures with only the empty assignment being reconciled. + assertEquals(1.0d, getMetricValue(metrics, rebalanceMetricsManager.rebalanceTotal)); + assertEquals(0.0d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceTotal)); + } + + @Test + public void testRebalanceMetricsOnSuccessfulRebalance() { + MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); + ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(new Assignment()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); + mockOwnedPartition(membershipManager, Uuid.randomUuid(), "topic1"); + + CompletableFuture commitResult = mockRevocationNoCallbacks(true); + + receiveEmptyAssignment(membershipManager); + long reconciliationDurationMs = 1234; + time.sleep(reconciliationDurationMs); + + membershipManager.poll(time.milliseconds()); + // Complete commit request to complete the callback invocation + commitResult.complete(null); + + assertEquals((double) reconciliationDurationMs, getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyTotal)); + assertEquals((double) reconciliationDurationMs, getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyAvg)); + assertEquals((double) reconciliationDurationMs, getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyMax)); + assertEquals(1d, getMetricValue(metrics, rebalanceMetricsManager.rebalanceTotal)); + assertEquals(120d, 1d, (double) getMetricValue(metrics, rebalanceMetricsManager.rebalanceRatePerHour)); + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceRate)); + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceTotal)); + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.lastRebalanceSecondsAgo)); + } + + @Test + public void testRebalanceMetricsForMultipleReconciliations() { + MembershipManagerImpl membershipManager = createMemberInStableState(); + ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); + + String topicName = "topic1"; + Uuid topicId = Uuid.randomUuid(); + + SleepyRebalanceListener listener = new SleepyRebalanceListener(1453, time); + when(subscriptionState.assignedPartitions()).thenReturn(Collections.emptySet()); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); + when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); + doNothing().when(subscriptionState).markPendingRevocation(anySet()); + when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); + + when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); + receiveAssignment(topicId, Arrays.asList(0, 1), membershipManager); + + membershipManager.poll(time.milliseconds()); + + // assign partitions + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + topicPartitions(topicName, 0, 1), + true + ); + + long firstRebalanaceTimesMs = listener.sleepMs; + listener.reset(); + + // ack + membershipManager.onHeartbeatRequestSent(); + + // revoke all + when(subscriptionState.assignedPartitions()).thenReturn(topicPartitions(topicName, 0, 1)); + receiveAssignment(topicId, Collections.singletonList(2), membershipManager); + + membershipManager.poll(time.milliseconds()); + + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, + topicPartitions(topicName, 0, 1), + true + ); + + // assign new partition 2 + performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + topicPartitions(topicName, 2), + true + ); + membershipManager.onHeartbeatRequestSent(); + + long secondRebalanceMs = listener.sleepMs; + long total = firstRebalanaceTimesMs + secondRebalanceMs; + double avg = total / 3.0d; + long max = Math.max(firstRebalanaceTimesMs, secondRebalanceMs); + assertEquals((double) total, getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyTotal)); + assertEquals(avg, (double) getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyAvg), 1d); + assertEquals((double) max, getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyMax)); + assertEquals(3d, getMetricValue(metrics, rebalanceMetricsManager.rebalanceTotal)); + // rate is not tested because it is subject to Rate implementation + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceRate)); + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceTotal)); + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.lastRebalanceSecondsAgo)); + + } + + @Test + public void testRebalanceMetricsOnFailedRebalance() { + MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); + ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(new Assignment()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); + + Uuid topicId = Uuid.randomUuid(); + + receiveAssignment(topicId, Arrays.asList(0, 1), membershipManager); + + // sleep for an arbitrary amount + time.sleep(2300); + + assertTrue(rebalanceMetricsManager.rebalanceStarted()); + membershipManager.onHeartbeatFailure(); + + assertEquals((double) 0, getMetricValue(metrics, rebalanceMetricsManager.rebalanceLatencyTotal)); + assertEquals(0d, getMetricValue(metrics, rebalanceMetricsManager.rebalanceTotal)); + assertEquals(120d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceRate)); + assertEquals(1d, getMetricValue(metrics, rebalanceMetricsManager.failedRebalanceTotal)); + assertEquals(-1d, getMetricValue(metrics, rebalanceMetricsManager.lastRebalanceSecondsAgo)); + } + + private Object getMetricValue(Metrics metrics, MetricName name) { + return metrics.metrics().get(name).metricValue(); + } + private MembershipManagerImpl mockMemberSuccessfullyReceivesAndAcksAssignment( Uuid topicId, String topicName, List partitions) { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); @@ -1992,6 +2357,57 @@ private CompletableFuture mockNewAssignmentAndRevocationStuckOnCommit( return commitResult; } + private ConsumerRebalanceListenerCallbackCompletedEvent mockNewAssignmentStuckOnPartitionsRevokedCallback( + MembershipManagerImpl membershipManager, Uuid topicId, String topicName, + List partitions, TopicPartition ownedPartition, ConsumerRebalanceListenerInvoker invoker) { + doNothing().when(subscriptionState).markPendingRevocation(anySet()); + CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener(); + when(subscriptionState.assignedPartitions()).thenReturn(Collections.singleton(ownedPartition)); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); + when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); + when(commitRequestManager.autoCommitEnabled()).thenReturn(false); + + when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); + receiveAssignment(topicId, partitions, membershipManager); + membershipManager.poll(time.milliseconds()); + verifyReconciliationTriggered(membershipManager); + clearInvocations(membershipManager); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + + return performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, + topicPartitions(ownedPartition.topic(), ownedPartition.partition()), + false + ); + } + + private ConsumerRebalanceListenerCallbackCompletedEvent mockNewAssignmentStuckOnPartitionsAssignedCallback( + MembershipManagerImpl membershipManager, Uuid topicId, String topicName, int newPartition, + ConsumerRebalanceListenerInvoker invoker) { + CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener(); + when(subscriptionState.assignedPartitions()).thenReturn(Collections.emptySet()); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); + when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); + when(commitRequestManager.autoCommitEnabled()).thenReturn(false); + + when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); + receiveAssignment(topicId, Collections.singletonList(newPartition), membershipManager); + membershipManager.poll(time.milliseconds()); + verifyReconciliationTriggered(membershipManager); + clearInvocations(membershipManager); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + + return performCallback( + membershipManager, + invoker, + ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, + topicPartitions(topicName, newPartition), + false + ); + } + private void verifyReconciliationTriggered(MembershipManagerImpl membershipManager) { verify(membershipManager).markReconciliationInProgress(); assertEquals(MemberState.RECONCILING, membershipManager.state()); @@ -2013,7 +2429,7 @@ private void verifyReconciliationTriggeredAndCompleted(MembershipManagerImpl mem List expectedTopicPartitions = buildTopicPartitions(expectedAssignment); verify(subscriptionState).assignFromSubscribed(new HashSet<>(expectedTopicPartitions)); Map> assignmentByTopicId = assignmentByTopicId(expectedAssignment); - assertEquals(assignmentByTopicId, membershipManager.currentAssignment()); + assertEquals(assignmentByTopicId, membershipManager.currentAssignment().partitions); // The auto-commit interval should be reset (only once), when the reconciliation completes verify(commitRequestManager).resetAutoCommitTimer(); @@ -2041,7 +2457,7 @@ private CompletableFuture mockRevocationNoCallbacks(boolean withAutoCommit if (withAutoCommit) { when(commitRequestManager.autoCommitEnabled()).thenReturn(true); CompletableFuture commitResult = new CompletableFuture<>(); - when(commitRequestManager.maybeAutoCommitSyncNow(anyLong())).thenReturn(commitResult); + when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation(anyLong())).thenReturn(commitResult); return commitResult; } else { return CompletableFuture.completedFuture(null); @@ -2064,7 +2480,7 @@ private void testRevocationCompleted(MembershipManagerImpl membershipManager, List expectedCurrentAssignment) { assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); Map> assignmentByTopicId = assignmentByTopicId(expectedCurrentAssignment); - assertEquals(assignmentByTopicId, membershipManager.currentAssignment()); + assertEquals(assignmentByTopicId, membershipManager.currentAssignment().partitions); assertFalse(membershipManager.reconciliationInProgress()); verify(subscriptionState).markPendingRevocation(anySet()); @@ -2087,7 +2503,9 @@ private void mockOwnedPartitionAndAssignmentReceived(MembershipManagerImpl membe String topicName, Collection previouslyOwned) { when(subscriptionState.assignedPartitions()).thenReturn(getTopicPartitions(previouslyOwned)); - membershipManager.updateCurrentAssignment(new HashSet<>(previouslyOwned)); + HashMap> partitionsByTopicId = new HashMap<>(); + partitionsByTopicId.put(topicId, new TreeSet<>(previouslyOwned.stream().map(TopicIdPartition::partition).collect(Collectors.toSet()))); + membershipManager.updateAssignment(partitionsByTopicId); when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); when(subscriptionState.rebalanceListener()).thenReturn(Optional.empty()).thenReturn(Optional.empty()); @@ -2102,32 +2520,32 @@ private Set getTopicPartitions(Collection topi private void mockOwnedPartition(MembershipManagerImpl membershipManager, Uuid topicId, String topic) { int partition = 0; TopicPartition previouslyOwned = new TopicPartition(topic, partition); - membershipManager.updateCurrentAssignment( - Collections.singleton(new TopicIdPartition(topicId, new TopicPartition(topic, partition)))); + membershipManager.updateAssignment(mkMap(mkEntry(topicId, new TreeSet<>(Collections.singletonList(partition))))); when(subscriptionState.assignedPartitions()).thenReturn(Collections.singleton(previouslyOwned)); when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); } - private MembershipManagerImpl mockJoinAndReceiveAssignment(boolean expectSubscriptionUpdated) { - return mockJoinAndReceiveAssignment(expectSubscriptionUpdated, createAssignment(expectSubscriptionUpdated)); + private MembershipManagerImpl mockJoinAndReceiveAssignment(boolean triggerReconciliation) { + return mockJoinAndReceiveAssignment(triggerReconciliation, createAssignment(triggerReconciliation)); } - private MembershipManagerImpl mockJoinAndReceiveAssignment(boolean expectSubscriptionUpdated, + private MembershipManagerImpl mockJoinAndReceiveAssignment(boolean triggerReconciliation, ConsumerGroupHeartbeatResponseData.Assignment assignment) { MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(assignment); when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); when(subscriptionState.rebalanceListener()).thenReturn(Optional.empty()).thenReturn(Optional.empty()); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); - membershipManager.poll(time.milliseconds()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); - if (expectSubscriptionUpdated) { + if (triggerReconciliation) { + membershipManager.poll(time.milliseconds()); verify(subscriptionState).assignFromSubscribed(anyCollection()); } else { verify(subscriptionState, never()).assignFromSubscribed(anyCollection()); } + clearInvocations(membershipManager, commitRequestManager); return membershipManager; } @@ -2136,10 +2554,18 @@ private MembershipManagerImpl createMemberInStableState() { } private MembershipManagerImpl createMemberInStableState(String groupInstanceId) { - MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(groupInstanceId); - ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(null); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + MembershipManagerImpl membershipManager = createMembershipManagerJoiningGroup(groupInstanceId, null); + ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(new Assignment()); + when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); + when(subscriptionState.rebalanceListener()).thenReturn(Optional.empty()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); + assertEquals(MemberState.RECONCILING, membershipManager.state()); + membershipManager.poll(time.milliseconds()); + assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); + membershipManager.onHeartbeatRequestSent(); assertEquals(MemberState.STABLE, membershipManager.state()); + + clearInvocations(subscriptionState, membershipManager, commitRequestManager); return membershipManager; } @@ -2150,7 +2576,7 @@ private void receiveAssignment(Map> topicIdPartitionLis .setTopicId(tp.getKey()) .setPartitions(new ArrayList<>(tp.getValue()))).collect(Collectors.toList())); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(targetAssignment); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); } private void receiveAssignment(Uuid topicId, List partitions, MembershipManager membershipManager) { @@ -2160,10 +2586,18 @@ private void receiveAssignment(Uuid topicId, List partitions, Membershi .setTopicId(topicId) .setPartitions(partitions))); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(targetAssignment); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); } - private void receiveAssignmentAfterRejoin(Uuid topicId, List partitions, MembershipManager membershipManager) { + private Map> receiveAssignmentAfterRejoin(List partitions, + MembershipManagerImpl membershipManager, + Collection owned) { + // Get new assignment after rejoining. This should not trigger a reconciliation just + // yet because there is another one in progress, but should keep the new assignment ready + // to be reconciled next. + Uuid topicId = Uuid.randomUuid(); + mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId, "topic3", owned); + ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() .setTopicPartitions(Collections.singletonList( new ConsumerGroupHeartbeatResponseData.TopicPartitions() @@ -2171,7 +2605,30 @@ private void receiveAssignmentAfterRejoin(Uuid topicId, List partitions .setPartitions(partitions))); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponseWithBumpedEpoch(targetAssignment); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); + + verifyReconciliationNotTriggered(membershipManager); + Map> assignmentAfterRejoin = topicIdPartitionsMap(topicId, 5); + assertEquals(assignmentAfterRejoin, membershipManager.topicPartitionsAwaitingReconciliation()); + return assignmentAfterRejoin; + } + + private void assertInitialReconciliationDiscardedAfterRejoin( + MembershipManagerImpl membershipManager, + Map> assignmentAfterRejoin) { + verify(subscriptionState, never()).markPendingRevocation(any()); + verify(subscriptionState, never()).assignFromSubscribed(anyCollection()); + assertNotEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); + + // Assignment received after rejoining should be ready to reconcile on the next + // reconciliation loop. + assertEquals(assignmentAfterRejoin, membershipManager.topicPartitionsAwaitingReconciliation()); + + // Stale reconciliation should have been aborted and a new one should be triggered on the next poll. + assertFalse(membershipManager.reconciliationInProgress()); + clearInvocations(membershipManager); + membershipManager.poll(time.milliseconds()); + verify(membershipManager).markReconciliationInProgress(); } private void receiveEmptyAssignment(MembershipManager membershipManager) { @@ -2179,7 +2636,7 @@ private void receiveEmptyAssignment(MembershipManager membershipManager) { ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() .setTopicPartitions(Collections.emptyList()); ConsumerGroupHeartbeatResponse heartbeatResponse = createConsumerGroupHeartbeatResponse(targetAssignment); - membershipManager.onHeartbeatResponseReceived(heartbeatResponse.data()); + membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); } /** @@ -2218,7 +2675,7 @@ private void testLeaveGroupReleasesAssignmentAndResetsEpochToSendLeaveGroup(Memb assertFalse(leaveResult.isCompletedExceptionally()); assertEquals(MEMBER_ID, membershipManager.memberId()); assertEquals(-1, membershipManager.memberEpoch()); - assertTrue(membershipManager.currentAssignment().isEmpty()); + assertTrue(membershipManager.currentAssignment().isNone()); verify(subscriptionState).assignFromSubscribed(Collections.emptySet()); } @@ -2261,7 +2718,6 @@ private ConsumerRebalanceListenerCallbackCompletedEvent mockFencedMemberStuckOnU when(subscriptionState.assignedPartitions()).thenReturn(Collections.singleton(ownedPartition)); when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); - // doNothing().when(subscriptionState).markPendingRevocation(anySet()); when(commitRequestManager.autoCommitEnabled()).thenReturn(false); membershipManager.transitionToFenced(); return performCallback( @@ -2307,9 +2763,9 @@ private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponseWithB .setAssignment(assignment)); } - private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponseWithError() { + private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponseWithError(Errors error) { return new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() - .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()) + .setErrorCode(error.code()) .setMemberId(MEMBER_ID) .setMemberEpoch(5)); } @@ -2341,7 +2797,7 @@ private MembershipManagerImpl memberJoinWithAssignment() { when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, "topic")); receiveAssignment(topicId, Collections.singletonList(0), membershipManager); membershipManager.onHeartbeatRequestSent(); - assertFalse(membershipManager.currentAssignment().isEmpty()); + assertFalse(membershipManager.currentAssignment().isNone()); return membershipManager; } @@ -2356,4 +2812,29 @@ private static Stream notInGroupStates() { Arguments.of(MemberState.STALE)); } + private static class SleepyRebalanceListener implements ConsumerRebalanceListener { + private long sleepMs; + private final long sleepDurationMs; + private final Time time; + SleepyRebalanceListener(long sleepDurationMs, Time time) { + this.sleepDurationMs = sleepDurationMs; + this.time = time; + } + + @Override + public void onPartitionsRevoked(Collection partitions) { + sleepMs += sleepDurationMs; + time.sleep(sleepDurationMs); + } + + @Override + public void onPartitionsAssigned(Collection partitions) { + sleepMs += sleepDurationMs; + time.sleep(sleepDurationMs); + } + + public void reset() { + sleepMs = 0; + } + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherTest.java index a92c4ede8a..596b549dd5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherTest.java @@ -87,6 +87,7 @@ import static org.apache.kafka.test.TestUtils.assertOptional; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -462,7 +463,7 @@ private boolean listOffsetMatchesExpectedReset( OffsetResetStrategy strategy, AbstractRequest request ) { - assertTrue(request instanceof ListOffsetsRequest); + assertInstanceOf(ListOffsetsRequest.class, request); ListOffsetsRequest req = (ListOffsetsRequest) request; assertEquals(singleton(tp.topic()), req.data().topics().stream() diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java index 5ca034d636..582efdfe76 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java @@ -24,7 +24,7 @@ import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; -import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.ClusterResource; import org.apache.kafka.common.IsolationLevel; @@ -553,8 +553,8 @@ public void testResetOffsetsAuthorizationFailure() { assertNotNull(event); // Check that the event itself is of the expected type - assertInstanceOf(ErrorBackgroundEvent.class, event); - ErrorBackgroundEvent errorEvent = (ErrorBackgroundEvent) event; + assertInstanceOf(ErrorEvent.class, event); + ErrorEvent errorEvent = (ErrorEvent) event; assertNotNull(errorEvent.error()); // Check that the error held in the event is of the expected type @@ -835,7 +835,7 @@ private ClientResponse buildOffsetsForLeaderEpochResponse( final int endOffset) { AbstractRequest abstractRequest = request.requestBuilder().build(); - assertTrue(abstractRequest instanceof OffsetsForLeaderEpochRequest); + assertInstanceOf(OffsetsForLeaderEpochRequest.class, abstractRequest); OffsetsForLeaderEpochRequest offsetsForLeaderEpochRequest = (OffsetsForLeaderEpochRequest) abstractRequest; OffsetForLeaderEpochResponseData data = new OffsetForLeaderEpochResponseData(); partitions.forEach(tp -> { @@ -870,7 +870,7 @@ private ClientResponse buildOffsetsForLeaderEpochResponseWithErrors( final Map partitionErrors) { AbstractRequest abstractRequest = request.requestBuilder().build(); - assertTrue(abstractRequest instanceof OffsetsForLeaderEpochRequest); + assertInstanceOf(OffsetsForLeaderEpochRequest.class, abstractRequest); OffsetsForLeaderEpochRequest offsetsForLeaderEpochRequest = (OffsetsForLeaderEpochRequest) abstractRequest; OffsetForLeaderEpochResponseData data = new OffsetForLeaderEpochResponseData(); partitionErrors.keySet().forEach(tp -> { @@ -931,7 +931,7 @@ private ClientResponse buildClientResponse( final boolean disconnected, final AuthenticationException authenticationException) { AbstractRequest abstractRequest = request.requestBuilder().build(); - assertTrue(abstractRequest instanceof ListOffsetsRequest); + assertInstanceOf(ListOffsetsRequest.class, abstractRequest); ListOffsetsRequest offsetFetchRequest = (ListOffsetsRequest) abstractRequest; ListOffsetsResponse response = buildListOffsetsResponse(topicResponses); return new ClientResponse( diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java new file mode 100644 index 0000000000..640c7e98e4 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.ApiVersions; +import org.apache.kafka.clients.GroupRebalanceConfig; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.junit.jupiter.api.Test; + +import java.util.Optional; +import java.util.Properties; + +import static org.apache.kafka.test.TestUtils.requiredConsumerConfig; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; + +public class RequestManagersTest { + + @Test + public void testMemberStateListenerRegistered() { + + final MemberStateListener listener = (memberEpoch, memberId) -> { }; + + final Properties properties = requiredConsumerConfig(); + properties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroup"); + final ConsumerConfig config = new ConsumerConfig(properties); + final GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( + config, + GroupRebalanceConfig.ProtocolType.CONSUMER + ); + final RequestManagers requestManagers = RequestManagers.supplier( + new MockTime(), + new LogContext(), + mock(BackgroundEventHandler.class), + mock(ConsumerMetadata.class), + mock(SubscriptionState.class), + mock(FetchBuffer.class), + config, + groupRebalanceConfig, + mock(ApiVersions.class), + mock(FetchMetricsManager.class), + () -> mock(NetworkClientDelegate.class), + Optional.empty(), + new Metrics(), + mock(OffsetCommitCallbackInvoker.class), + listener + ).get(); + requestManagers.membershipManager.ifPresent( + membershipManager -> assertTrue(((MembershipManagerImpl) membershipManager).stateListeners().contains(listener)) + ); + } +} \ No newline at end of file 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 fe6cc5f028..e54bb36e17 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 @@ -39,6 +39,7 @@ import java.util.HashSet; import java.util.Optional; import java.util.Set; +import java.util.function.LongSupplier; import java.util.regex.Pattern; import static java.util.Collections.singleton; @@ -895,4 +896,75 @@ public void nullPositionLagOnNoPosition() { assertNull(state.partitionLag(tp0, IsolationLevel.READ_COMMITTED)); } + @Test + public void testPositionOrNull() { + state.assignFromUser(Collections.singleton(tp0)); + final TopicPartition unassignedPartition = new TopicPartition("unassigned", 0); + state.seek(tp0, 5); + + assertEquals(5, state.positionOrNull(tp0).offset); + assertNull(state.positionOrNull(unassignedPartition)); + } + + @Test + public void testTryUpdatingHighWatermark() { + state.assignFromUser(Collections.singleton(tp0)); + final TopicPartition unassignedPartition = new TopicPartition("unassigned", 0); + + final long highWatermark = 10L; + assertTrue(state.tryUpdatingHighWatermark(tp0, highWatermark)); + assertEquals(highWatermark, state.partitionEndOffset(tp0, IsolationLevel.READ_UNCOMMITTED)); + assertFalse(state.tryUpdatingHighWatermark(unassignedPartition, highWatermark)); + } + + @Test + public void testTryUpdatingLogStartOffset() { + state.assignFromUser(Collections.singleton(tp0)); + final TopicPartition unassignedPartition = new TopicPartition("unassigned", 0); + final long position = 25; + state.seek(tp0, position); + + final long logStartOffset = 10L; + assertTrue(state.tryUpdatingLogStartOffset(tp0, logStartOffset)); + assertEquals(position - logStartOffset, state.partitionLead(tp0)); + assertFalse(state.tryUpdatingLogStartOffset(unassignedPartition, logStartOffset)); + } + + @Test + public void testTryUpdatingLastStableOffset() { + state.assignFromUser(Collections.singleton(tp0)); + final TopicPartition unassignedPartition = new TopicPartition("unassigned", 0); + + final long lastStableOffset = 10L; + assertTrue(state.tryUpdatingLastStableOffset(tp0, lastStableOffset)); + assertEquals(lastStableOffset, state.partitionEndOffset(tp0, IsolationLevel.READ_COMMITTED)); + assertFalse(state.tryUpdatingLastStableOffset(unassignedPartition, lastStableOffset)); + } + + @Test + public void testTryUpdatingPreferredReadReplica() { + state.assignFromUser(Collections.singleton(tp0)); + final TopicPartition unassignedPartition = new TopicPartition("unassigned", 0); + + final int preferredReadReplicaId = 10; + final LongSupplier expirationTimeMs = () -> System.currentTimeMillis() + 60000L; + assertTrue(state.tryUpdatingPreferredReadReplica(tp0, preferredReadReplicaId, expirationTimeMs)); + assertEquals(Optional.of(preferredReadReplicaId), state.preferredReadReplica(tp0, System.currentTimeMillis())); + assertFalse(state.tryUpdatingPreferredReadReplica(unassignedPartition, preferredReadReplicaId, expirationTimeMs)); + assertEquals(Optional.empty(), state.preferredReadReplica(unassignedPartition, System.currentTimeMillis())); + } + + @Test + public void testRequestOffsetResetIfPartitionAssigned() { + state.assignFromUser(Collections.singleton(tp0)); + final TopicPartition unassignedPartition = new TopicPartition("unassigned", 0); + + state.requestOffsetResetIfPartitionAssigned(tp0); + + assertTrue(state.isOffsetResetNeeded(tp0)); + + state.requestOffsetResetIfPartitionAssigned(unassignedPartition); + + assertThrows(IllegalStateException.class, () -> state.isOffsetResetNeeded(unassignedPartition)); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java index c7b2315060..3f2b2c3d98 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java @@ -56,6 +56,7 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.spy; @@ -222,7 +223,7 @@ private ClientResponse buildTopicMetadataClientResponse( final String topic, final Errors error) { AbstractRequest abstractRequest = request.requestBuilder().build(); - assertTrue(abstractRequest instanceof MetadataRequest); + assertInstanceOf(MetadataRequest.class, abstractRequest); MetadataRequest metadataRequest = (MetadataRequest) abstractRequest; Cluster cluster = mockCluster(3, 0); List topics = new ArrayList<>(); @@ -248,7 +249,7 @@ private ClientResponse buildAllTopicsMetadataClientResponse( final NetworkClientDelegate.UnsentRequest request, final Errors error) { AbstractRequest abstractRequest = request.requestBuilder().build(); - assertTrue(abstractRequest instanceof MetadataRequest); + assertInstanceOf(MetadataRequest.class, abstractRequest); MetadataRequest metadataRequest = (MetadataRequest) abstractRequest; Cluster cluster = mockCluster(3, 0); List topics = new ArrayList<>(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/WakeupTriggerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/WakeupTriggerTest.java index 235ec78168..a1f15d3d73 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/WakeupTriggerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/WakeupTriggerTest.java @@ -28,12 +28,13 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; @MockitoSettings(strictness = Strictness.STRICT_STUBS) @@ -133,16 +134,46 @@ public void testManualTriggerWhenWakeupCalledAndFetchActionSet() { } } + @Test + public void testDisableWakeupWithoutPendingTask() { + wakeupTrigger.disableWakeups(); + wakeupTrigger.wakeup(); + assertDoesNotThrow(() -> wakeupTrigger.maybeTriggerWakeup()); + } + + @Test + public void testDisableWakeupWithPendingTask() { + final CompletableFuture future = new CompletableFuture<>(); + wakeupTrigger.disableWakeups(); + wakeupTrigger.setActiveTask(future); + wakeupTrigger.wakeup(); + assertFalse(future.isCompletedExceptionally()); + assertDoesNotThrow(() -> wakeupTrigger.maybeTriggerWakeup()); + } + + @Test + public void testDisableWakeupWithFetchAction() { + try (final FetchBuffer fetchBuffer = mock(FetchBuffer.class)) { + wakeupTrigger.disableWakeups(); + wakeupTrigger.setFetchAction(fetchBuffer); + wakeupTrigger.wakeup(); + verify(fetchBuffer, never()).wakeup(); + } + } + + @Test + public void testDisableWakeupPreservedByClearTask() { + final CompletableFuture future = new CompletableFuture<>(); + wakeupTrigger.disableWakeups(); + wakeupTrigger.setActiveTask(future); + wakeupTrigger.clearTask(); + wakeupTrigger.wakeup(); + assertDoesNotThrow(() -> wakeupTrigger.maybeTriggerWakeup()); + } + private void assertWakeupExceptionIsThrown(final CompletableFuture future) { assertTrue(future.isCompletedExceptionally()); - try { - future.get(DEFAULT_TIMEOUT_MS, TimeUnit.MILLISECONDS); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof WakeupException); - return; - } catch (Exception e) { - fail("The task should throw an ExecutionException but got:" + e); - } - fail("The task should throw an ExecutionException"); + assertInstanceOf(WakeupException.class, + assertThrows(ExecutionException.class, () -> future.get(DEFAULT_TIMEOUT_MS, TimeUnit.MILLISECONDS)).getCause()); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 8ea8cb7a72..b23660e546 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -27,6 +27,9 @@ import org.apache.kafka.clients.consumer.internals.RequestManagers; import org.apache.kafka.clients.consumer.internals.TopicMetadataRequestManager; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -36,7 +39,6 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; -import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -44,6 +46,7 @@ import static org.mockito.Mockito.when; public class ApplicationEventProcessorTest { + private final Time time = new MockTime(1); private ApplicationEventProcessor processor; private BlockingQueue applicationEventQueue = mock(BlockingQueue.class); private RequestManagers requestManagers; @@ -92,22 +95,14 @@ public void setup() { public void testPrepClosingCommitEvents() { List results = mockCommitResults(); doReturn(new NetworkClientDelegate.PollResult(100, results)).when(commitRequestManager).pollOnClose(); - processor.process(new CommitOnCloseApplicationEvent()); + processor.process(new CommitOnCloseEvent()); verify(commitRequestManager).signalClose(); } - @Test - public void testExpirationCalculation() { - assertEquals(Long.MAX_VALUE, processor.getExpirationTimeForTimeout(Long.MAX_VALUE)); - assertEquals(Long.MAX_VALUE, processor.getExpirationTimeForTimeout(Long.MAX_VALUE - 1)); - long timeout = processor.getExpirationTimeForTimeout(1000); - assertTrue(timeout > 0); - assertTrue(timeout < Long.MAX_VALUE); - } - @Test public void testPrepClosingLeaveGroupEvent() { - LeaveOnCloseApplicationEvent event = new LeaveOnCloseApplicationEvent(); + Timer timer = time.timer(100); + LeaveOnCloseEvent event = new LeaveOnCloseEvent(timer); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); when(membershipManager.leaveGroup()).thenReturn(CompletableFuture.completedFuture(null)); processor.process(event); 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 3cb6fc0782..781aaaaf31 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 @@ -127,6 +127,7 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -491,7 +492,7 @@ public void testExplicitlyOnlyEnableJmxReporter() { props.setProperty(ProducerConfig.ENABLE_METRICS_PUSH_CONFIG, "false"); KafkaProducer producer = new KafkaProducer<>(props, new StringSerializer(), new StringSerializer()); assertEquals(1, producer.metrics.reporters().size()); - assertTrue(producer.metrics.reporters().get(0) instanceof JmxReporter); + assertInstanceOf(JmxReporter.class, producer.metrics.reporters().get(0)); producer.close(); } @@ -503,7 +504,7 @@ public void testExplicitlyOnlyEnableClientTelemetryReporter() { props.setProperty(ProducerConfig.AUTO_INCLUDE_JMX_REPORTER_CONFIG, "false"); KafkaProducer producer = new KafkaProducer<>(props, new StringSerializer(), new StringSerializer()); assertEquals(1, producer.metrics.reporters().size()); - assertTrue(producer.metrics.reporters().get(0) instanceof ClientTelemetryReporter); + assertInstanceOf(ClientTelemetryReporter.class, producer.metrics.reporters().get(0)); producer.close(); } @@ -692,7 +693,7 @@ public void shouldCloseProperlyAndThrowIfInterrupted() throws Exception { TestUtils.waitForCondition(() -> closeException.get() != null, "InterruptException did not occur within timeout."); - assertTrue(closeException.get() instanceof InterruptException, "Expected exception not thrown " + closeException); + assertInstanceOf(InterruptException.class, closeException.get(), "Expected exception not thrown " + closeException); } finally { executor.shutdownNow(); } @@ -848,9 +849,7 @@ public void testMetadataTimeoutWithMissingTopic(boolean isIdempotenceEnabled) th verify(metadata, times(4)).awaitUpdate(anyInt(), anyLong()); verify(metadata, times(5)).fetch(); try { - future.get(); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof TimeoutException); + assertInstanceOf(TimeoutException.class, assertThrows(ExecutionException.class, future::get).getCause()); } finally { producer.close(Duration.ofMillis(0)); } @@ -917,9 +916,7 @@ public void testMetadataTimeoutWithPartitionOutOfRange(boolean isIdempotenceEnab verify(metadata, times(4)).awaitUpdate(anyInt(), anyLong()); verify(metadata, times(5)).fetch(); try { - future.get(); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof TimeoutException); + assertInstanceOf(TimeoutException.class, assertThrows(ExecutionException.class, future::get).getCause()); } finally { producer.close(Duration.ofMillis(0)); } 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 9141fd51cb..61950cce34 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 @@ -43,6 +43,7 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -262,7 +263,7 @@ public void shouldThrowOnSendIfProducerGotFenced() { producer.initTransactions(); producer.fenceProducer(); Throwable e = assertThrows(KafkaException.class, () -> producer.send(null)); - assertTrue(e.getCause() instanceof ProducerFencedException, "The root cause of the exception should be ProducerFenced"); + assertInstanceOf(ProducerFencedException.class, e.getCause(), "The root cause of the exception should be ProducerFenced"); } @Test 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 9af3aff8bb..b9c734a2cb 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 @@ -42,6 +42,7 @@ import org.apache.kafka.common.errors.TransactionAbortedException; import org.apache.kafka.common.errors.UnsupportedForMessageFormatException; import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.errors.AbortableTransactionException; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.message.AddPartitionsToTxnResponseData; import org.apache.kafka.common.message.ApiMessageType; @@ -114,6 +115,7 @@ import static org.apache.kafka.clients.producer.internals.ProducerTestUtils.runUntil; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -733,7 +735,7 @@ public void testClusterAuthorizationExceptionInInitProducerIdRequest() throws Ex prepareAndReceiveInitProducerId(producerId, Errors.CLUSTER_AUTHORIZATION_FAILED); assertFalse(transactionManager.hasProducerId()); assertTrue(transactionManager.hasError()); - assertTrue(transactionManager.lastError() instanceof ClusterAuthorizationException); + assertInstanceOf(ClusterAuthorizationException.class, transactionManager.lastError()); assertEquals(-1, transactionManager.producerIdAndEpoch().epoch); assertSendFailure(ClusterAuthorizationException.class); @@ -768,11 +770,7 @@ public void testCanRetryWithoutIdempotence() throws Exception { }, produceResponse(tp0, -1L, Errors.TOPIC_AUTHORIZATION_FAILED, 0)); sender.runOnce(); assertTrue(future.isDone()); - try { - future.get(); - } catch (Exception e) { - assertTrue(e.getCause() instanceof TopicAuthorizationException); - } + assertInstanceOf(TopicAuthorizationException.class, assertThrows(Exception.class, future::get).getCause()); } @Test @@ -2539,12 +2537,10 @@ public void testInflightBatchesExpireOnDeliveryTimeout() throws InterruptedExcep time.sleep(deliveryTimeoutMs); sender.runOnce(); // receive first response assertEquals(0, sender.inFlightBatches(tp0).size(), "Expect zero in-flight batch in accumulator"); - try { - request.get(); - fail("The expired batch should throw a TimeoutException"); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof TimeoutException); - } + assertInstanceOf( + TimeoutException.class, + assertThrows(ExecutionException.class, request::get).getCause(), + "The expired batch should throw a TimeoutException"); } @Test @@ -2578,10 +2574,10 @@ public void testRecordErrorPropagatedToApplication() throws InterruptedException KafkaException exception = TestUtils.assertFutureThrows(future, KafkaException.class); Integer index = futureEntry.getKey(); if (index == 0 || index == 2) { - assertTrue(exception instanceof InvalidRecordException); + assertInstanceOf(InvalidRecordException.class, exception); assertEquals(index.toString(), exception.getMessage()); } else if (index == 3) { - assertTrue(exception instanceof InvalidRecordException); + assertInstanceOf(InvalidRecordException.class, exception); assertEquals(Errors.INVALID_RECORD.message(), exception.getMessage()); } else { assertEquals(KafkaException.class, exception.getClass()); @@ -2722,10 +2718,10 @@ public void testExpiredBatchesInMultiplePartitions() throws Exception { assertEquals(0, sender.inFlightBatches(tp0).size(), "Expect zero in-flight batch in accumulator"); ExecutionException e = assertThrows(ExecutionException.class, request1::get); - assertTrue(e.getCause() instanceof TimeoutException); + assertInstanceOf(TimeoutException.class, e.getCause()); e = assertThrows(ExecutionException.class, request2::get); - assertTrue(e.getCause() instanceof TimeoutException); + assertInstanceOf(TimeoutException.class, e.getCause()); } @Test @@ -3151,6 +3147,45 @@ public void testInvalidTxnStateIsAnAbortableError() throws Exception { txnManager.beginTransaction(); } + + @Test + public void testAbortableTxnExceptionIsAnAbortableError() throws Exception { + ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); + apiVersions.update("0", NodeApiVersions.create(ApiKeys.INIT_PRODUCER_ID.id, (short) 0, (short) 3)); + TransactionManager txnManager = new TransactionManager(logContext, "textAbortableTxnException", 60000, 100, apiVersions); + + setupWithTransactionState(txnManager); + doInitTransactions(txnManager, producerIdAndEpoch); + + txnManager.beginTransaction(); + txnManager.maybeAddPartition(tp0); + client.prepareResponse(buildAddPartitionsToTxnResponseData(0, Collections.singletonMap(tp0, Errors.NONE))); + sender.runOnce(); + + Future request = appendToAccumulator(tp0); + sender.runOnce(); // send request + sendIdempotentProducerResponse(0, tp0, Errors.ABORTABLE_TRANSACTION, -1); + + // Return AbortableTransactionException error. It should be abortable. + sender.runOnce(); + assertFutureFailure(request, AbortableTransactionException.class); + assertTrue(txnManager.hasAbortableError()); + TransactionalRequestResult result = txnManager.beginAbort(); + sender.runOnce(); + + // Once the transaction is aborted, we should be able to begin a new one. + respondToEndTxn(Errors.NONE); + sender.runOnce(); + assertTrue(txnManager::isInitializing); + prepareInitProducerResponse(Errors.NONE, producerIdAndEpoch.producerId, producerIdAndEpoch.epoch); + sender.runOnce(); + assertTrue(txnManager::isReady); + + assertTrue(result.isSuccessful()); + result.await(); + + txnManager.beginTransaction(); + } @Test public void testProducerBatchRetriesWhenPartitionLeaderChanges() throws Exception { Metrics m = new Metrics(); 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 b099ebfc41..4d8c445be0 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 @@ -40,6 +40,7 @@ import org.apache.kafka.common.errors.TransactionalIdAuthorizationException; import org.apache.kafka.common.errors.UnsupportedForMessageFormatException; import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.errors.AbortableTransactionException; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.message.AddOffsetsToTxnResponseData; @@ -109,6 +110,7 @@ import static java.util.Collections.singletonMap; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -883,7 +885,7 @@ public void testUnsupportedFindCoordinator() { runUntil(transactionManager::hasFatalError); assertTrue(transactionManager.hasFatalError()); - assertTrue(transactionManager.lastError() instanceof UnsupportedVersionException); + assertInstanceOf(UnsupportedVersionException.class, transactionManager.lastError()); } @Test @@ -902,7 +904,7 @@ public void testUnsupportedInitTransactions() { runUntil(transactionManager::hasFatalError); assertTrue(transactionManager.hasFatalError()); - assertTrue(transactionManager.lastError() instanceof UnsupportedVersionException); + assertInstanceOf(UnsupportedVersionException.class, transactionManager.lastError()); } @Test @@ -920,10 +922,10 @@ public void testUnsupportedForMessageFormatInTxnOffsetCommit() { prepareTxnOffsetCommitResponse(consumerGroupId, producerId, epoch, singletonMap(tp, Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT)); runUntil(transactionManager::hasError); - assertTrue(transactionManager.lastError() instanceof UnsupportedForMessageFormatException); + assertInstanceOf(UnsupportedForMessageFormatException.class, transactionManager.lastError()); assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); - assertTrue(sendOffsetsResult.error() instanceof UnsupportedForMessageFormatException); + assertInstanceOf(UnsupportedForMessageFormatException.class, sendOffsetsResult.error()); assertFatalError(UnsupportedForMessageFormatException.class); } @@ -954,10 +956,10 @@ public void testFencedInstanceIdInTxnOffsetCommitByGroupMetadata() { }, new TxnOffsetCommitResponse(0, singletonMap(tp, Errors.FENCED_INSTANCE_ID))); runUntil(transactionManager::hasError); - assertTrue(transactionManager.lastError() instanceof FencedInstanceIdException); + assertInstanceOf(FencedInstanceIdException.class, transactionManager.lastError()); assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); - assertTrue(sendOffsetsResult.error() instanceof FencedInstanceIdException); + assertInstanceOf(FencedInstanceIdException.class, sendOffsetsResult.error()); assertAbortableError(FencedInstanceIdException.class); } @@ -987,10 +989,10 @@ public void testUnknownMemberIdInTxnOffsetCommitByGroupMetadata() { }, new TxnOffsetCommitResponse(0, singletonMap(tp, Errors.UNKNOWN_MEMBER_ID))); runUntil(transactionManager::hasError); - assertTrue(transactionManager.lastError() instanceof CommitFailedException); + assertInstanceOf(CommitFailedException.class, transactionManager.lastError()); assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); - assertTrue(sendOffsetsResult.error() instanceof CommitFailedException); + assertInstanceOf(CommitFailedException.class, sendOffsetsResult.error()); assertAbortableError(CommitFailedException.class); } @@ -1022,10 +1024,10 @@ public void testIllegalGenerationInTxnOffsetCommitByGroupMetadata() { }, new TxnOffsetCommitResponse(0, singletonMap(tp, Errors.ILLEGAL_GENERATION))); runUntil(transactionManager::hasError); - assertTrue(transactionManager.lastError() instanceof CommitFailedException); + assertInstanceOf(CommitFailedException.class, transactionManager.lastError()); assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); - assertTrue(sendOffsetsResult.error() instanceof CommitFailedException); + assertInstanceOf(CommitFailedException.class, sendOffsetsResult.error()); assertAbortableError(CommitFailedException.class); } @@ -1128,7 +1130,7 @@ public void testTransactionalIdAuthorizationFailureInFindCoordinator() { runUntil(transactionManager::hasError); assertTrue(transactionManager.hasFatalError()); - assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); + assertInstanceOf(TransactionalIdAuthorizationException.class, transactionManager.lastError()); assertFalse(initPidResult.isSuccessful()); assertThrows(TransactionalIdAuthorizationException.class, initPidResult::await); assertFatalError(TransactionalIdAuthorizationException.class); @@ -1162,11 +1164,11 @@ public void testGroupAuthorizationFailureInFindCoordinator() { prepareFindCoordinatorResponse(Errors.GROUP_AUTHORIZATION_FAILED, false, CoordinatorType.GROUP, consumerGroupId); runUntil(transactionManager::hasError); - assertTrue(transactionManager.lastError() instanceof GroupAuthorizationException); + assertInstanceOf(GroupAuthorizationException.class, transactionManager.lastError()); runUntil(sendOffsetsResult::isCompleted); assertFalse(sendOffsetsResult.isSuccessful()); - assertTrue(sendOffsetsResult.error() instanceof GroupAuthorizationException); + assertInstanceOf(GroupAuthorizationException.class, sendOffsetsResult.error()); GroupAuthorizationException exception = (GroupAuthorizationException) sendOffsetsResult.error(); assertEquals(consumerGroupId, exception.groupId()); @@ -1191,10 +1193,10 @@ public void testGroupAuthorizationFailureInTxnOffsetCommit() { prepareTxnOffsetCommitResponse(consumerGroupId, producerId, epoch, singletonMap(tp1, Errors.GROUP_AUTHORIZATION_FAILED)); runUntil(transactionManager::hasError); - assertTrue(transactionManager.lastError() instanceof GroupAuthorizationException); + assertInstanceOf(GroupAuthorizationException.class, transactionManager.lastError()); assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); - assertTrue(sendOffsetsResult.error() instanceof GroupAuthorizationException); + assertInstanceOf(GroupAuthorizationException.class, sendOffsetsResult.error()); assertFalse(transactionManager.hasPendingOffsetCommits()); GroupAuthorizationException exception = (GroupAuthorizationException) sendOffsetsResult.error(); @@ -1215,10 +1217,10 @@ public void testTransactionalIdAuthorizationFailureInAddOffsetsToTxn() { prepareAddOffsetsToTxnResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, consumerGroupId, producerId, epoch); runUntil(transactionManager::hasError); - assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); + assertInstanceOf(TransactionalIdAuthorizationException.class, transactionManager.lastError()); assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); - assertTrue(sendOffsetsResult.error() instanceof TransactionalIdAuthorizationException); + assertInstanceOf(TransactionalIdAuthorizationException.class, sendOffsetsResult.error()); assertFatalError(TransactionalIdAuthorizationException.class); } @@ -1235,10 +1237,10 @@ public void testInvalidTxnStateFailureInAddOffsetsToTxn() { prepareAddOffsetsToTxnResponse(Errors.INVALID_TXN_STATE, consumerGroupId, producerId, epoch); runUntil(transactionManager::hasError); - assertTrue(transactionManager.lastError() instanceof InvalidTxnStateException); + assertInstanceOf(InvalidTxnStateException.class, transactionManager.lastError()); assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); - assertTrue(sendOffsetsResult.error() instanceof InvalidTxnStateException); + assertInstanceOf(InvalidTxnStateException.class, sendOffsetsResult.error()); assertFatalError(InvalidTxnStateException.class); } @@ -1260,10 +1262,10 @@ public void testTransactionalIdAuthorizationFailureInTxnOffsetCommit() { prepareTxnOffsetCommitResponse(consumerGroupId, producerId, epoch, singletonMap(tp, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED)); runUntil(transactionManager::hasError); - assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); + assertInstanceOf(TransactionalIdAuthorizationException.class, transactionManager.lastError()); assertTrue(sendOffsetsResult.isCompleted()); assertFalse(sendOffsetsResult.isSuccessful()); - assertTrue(sendOffsetsResult.error() instanceof TransactionalIdAuthorizationException); + assertInstanceOf(TransactionalIdAuthorizationException.class, sendOffsetsResult.error()); assertFatalError(TransactionalIdAuthorizationException.class); } @@ -1289,7 +1291,7 @@ public void testTopicAuthorizationFailureInAddPartitions() throws InterruptedExc prepareAddPartitionsToTxn(errors); runUntil(transactionManager::hasError); - assertTrue(transactionManager.lastError() instanceof TopicAuthorizationException); + assertInstanceOf(TopicAuthorizationException.class, transactionManager.lastError()); assertFalse(transactionManager.isPartitionPendingAdd(tp0)); assertFalse(transactionManager.isPartitionPendingAdd(tp1)); assertFalse(transactionManager.isPartitionAdded(tp0)); @@ -1351,7 +1353,7 @@ public void testCommitWithTopicAuthorizationFailureInAddPartitionsInFlight() thr assertTrue(commitResult.isCompleted()); TestUtils.assertFutureThrows(firstPartitionAppend, KafkaException.class); TestUtils.assertFutureThrows(secondPartitionAppend, KafkaException.class); - assertTrue(commitResult.error() instanceof TopicAuthorizationException); + assertInstanceOf(TopicAuthorizationException.class, commitResult.error()); } @Test @@ -1626,7 +1628,7 @@ public void testTransactionalIdAuthorizationFailureInAddPartitions() { prepareAddPartitionsToTxn(tp, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED); runUntil(transactionManager::hasError); - assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); + assertInstanceOf(TransactionalIdAuthorizationException.class, transactionManager.lastError()); assertFatalError(TransactionalIdAuthorizationException.class); } @@ -1642,7 +1644,7 @@ public void testInvalidTxnStateInAddPartitions() { prepareAddPartitionsToTxn(tp, Errors.INVALID_TXN_STATE); runUntil(transactionManager::hasError); - assertTrue(transactionManager.lastError() instanceof InvalidTxnStateException); + assertInstanceOf(InvalidTxnStateException.class, transactionManager.lastError()); assertFatalError(InvalidTxnStateException.class); } @@ -1866,7 +1868,7 @@ private void verifyProducerFenced(Future responseFuture) throws responseFuture.get(); fail("Expected to get a ExecutionException from the response"); } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof ProducerFencedException); + assertInstanceOf(ProducerFencedException.class, e.getCause()); } // make sure the exception was thrown directly from the follow-up calls. @@ -1932,13 +1934,13 @@ public void testInvalidProducerEpochFromProduce() throws InterruptedException { // First we will get an EndTxn for abort. assertNotNull(handler); - assertTrue(handler.requestBuilder() instanceof EndTxnRequest.Builder); + assertInstanceOf(EndTxnRequest.Builder.class, handler.requestBuilder()); handler = transactionManager.nextRequest(false); // Second we will see an InitPid for handling InvalidProducerEpoch. assertNotNull(handler); - assertTrue(handler.requestBuilder() instanceof InitProducerIdRequest.Builder); + assertInstanceOf(InitProducerIdRequest.Builder.class, handler.requestBuilder()); } @Test @@ -2433,7 +2435,7 @@ public void testSendOffsetWithGroupMetadataFailAsAutoDowngradeTxnCommitNotEnable assertTrue(addOffsetsResult.isCompleted()); assertFalse(addOffsetsResult.isSuccessful()); - assertTrue(addOffsetsResult.error() instanceof UnsupportedVersionException); + assertInstanceOf(UnsupportedVersionException.class, addOffsetsResult.error()); assertFatalError(UnsupportedVersionException.class); } @@ -2614,13 +2616,11 @@ public void testTransitionToAbortableErrorOnBatchExpiry() throws InterruptedExce runUntil(responseFuture::isDone); - try { - // make sure the produce was expired. - responseFuture.get(); - fail("Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof TimeoutException); - } + // make sure the produce was expired. + assertInstanceOf( + TimeoutException.class, + assertThrows(ExecutionException.class, responseFuture::get).getCause(), + "Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); assertTrue(transactionManager.hasAbortableError()); } @@ -2664,21 +2664,17 @@ public void testTransitionToAbortableErrorOnMultipleBatchExpiry() throws Interru runUntil(firstBatchResponse::isDone); runUntil(secondBatchResponse::isDone); - try { - // make sure the produce was expired. - firstBatchResponse.get(); - fail("Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof TimeoutException); - } + // make sure the produce was expired. + assertInstanceOf( + TimeoutException.class, + assertThrows(ExecutionException.class, firstBatchResponse::get).getCause(), + "Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); + // make sure the produce was expired. + assertInstanceOf( + TimeoutException.class, + assertThrows(ExecutionException.class, secondBatchResponse::get).getCause(), + "Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); - try { - // make sure the produce was expired. - secondBatchResponse.get(); - fail("Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof TimeoutException); - } assertTrue(transactionManager.hasAbortableError()); } @@ -2713,13 +2709,11 @@ public void testDropCommitOnBatchExpiry() throws InterruptedException { runUntil(responseFuture::isDone); // We should try to flush the produce, but expire it instead without sending anything. - try { - // make sure the produce was expired. - responseFuture.get(); - fail("Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof TimeoutException); - } + // make sure the produce was expired. + assertInstanceOf( + TimeoutException.class, + assertThrows(ExecutionException.class, responseFuture::get).getCause(), + "Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); runUntil(commitResult::isCompleted); // the commit shouldn't be completed without being sent since the produce request failed. assertFalse(commitResult.isSuccessful()); // the commit shouldn't succeed since the produce request failed. assertThrows(TimeoutException.class, commitResult::await); @@ -2784,13 +2778,11 @@ public void testTransitionToFatalErrorWhenRetriedBatchIsExpired() throws Interru runUntil(responseFuture::isDone); // We should try to flush the produce, but expire it instead without sending anything. - try { - // make sure the produce was expired. - responseFuture.get(); - fail("Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof TimeoutException); - } + // make sure the produce was expired. + assertInstanceOf( + TimeoutException.class, + assertThrows(ExecutionException.class, responseFuture::get).getCause(), + "Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); runUntil(commitResult::isCompleted); assertFalse(commitResult.isSuccessful()); // the commit should have been dropped. @@ -3523,6 +3515,126 @@ public void testForegroundInvalidStateTransitionIsRecoverable() { assertFalse(transactionManager.hasOngoingTransaction()); } + @Test + public void testAbortableTransactionExceptionInInitProducerId() { + TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); + prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); + runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); + assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); + + prepareInitPidResponse(Errors.ABORTABLE_TRANSACTION, false, producerId, RecordBatch.NO_PRODUCER_EPOCH); + runUntil(transactionManager::hasError); + assertTrue(initPidResult.isCompleted()); + assertFalse(initPidResult.isSuccessful()); + assertThrows(AbortableTransactionException.class, initPidResult::await); + assertAbortableError(AbortableTransactionException.class); + } + + @Test + public void testAbortableTransactionExceptionInAddPartitions() { + final TopicPartition tp = new TopicPartition("foo", 0); + + doInitTransactions(); + + transactionManager.beginTransaction(); + transactionManager.maybeAddPartition(tp); + + prepareAddPartitionsToTxn(tp, Errors.ABORTABLE_TRANSACTION); + runUntil(transactionManager::hasError); + assertTrue(transactionManager.lastError() instanceof AbortableTransactionException); + + assertAbortableError(AbortableTransactionException.class); + } + + @Test + public void testAbortableTransactionExceptionInFindCoordinator() { + doInitTransactions(); + + transactionManager.beginTransaction(); + TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( + singletonMap(new TopicPartition("foo", 0), new OffsetAndMetadata(39L)), new ConsumerGroupMetadata(consumerGroupId)); + + prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, producerId, epoch); + runUntil(() -> !transactionManager.hasPartitionsToAdd()); + + prepareFindCoordinatorResponse(Errors.ABORTABLE_TRANSACTION, false, CoordinatorType.GROUP, consumerGroupId); + runUntil(transactionManager::hasError); + assertTrue(transactionManager.lastError() instanceof AbortableTransactionException); + + runUntil(sendOffsetsResult::isCompleted); + assertFalse(sendOffsetsResult.isSuccessful()); + assertTrue(sendOffsetsResult.error() instanceof AbortableTransactionException); + + assertAbortableError(AbortableTransactionException.class); + } + + @Test + public void testAbortableTransactionExceptionInEndTxn() throws InterruptedException { + doInitTransactions(); + + transactionManager.beginTransaction(); + transactionManager.maybeAddPartition(tp0); + TransactionalRequestResult commitResult = transactionManager.beginCommit(); + + Future responseFuture = appendToAccumulator(tp0); + + assertFalse(responseFuture.isDone()); + prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, producerId); + prepareProduceResponse(Errors.NONE, producerId, epoch); + prepareEndTxnResponse(Errors.ABORTABLE_TRANSACTION, TransactionResult.COMMIT, producerId, epoch); + + runUntil(commitResult::isCompleted); + runUntil(responseFuture::isDone); + + assertThrows(KafkaException.class, commitResult::await); + assertFalse(commitResult.isSuccessful()); + assertTrue(commitResult.isAcked()); + + assertAbortableError(AbortableTransactionException.class); + } + + @Test + public void testAbortableTransactionExceptionInAddOffsetsToTxn() { + final TopicPartition tp = new TopicPartition("foo", 0); + + doInitTransactions(); + + transactionManager.beginTransaction(); + TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( + singletonMap(tp, new OffsetAndMetadata(39L)), new ConsumerGroupMetadata(consumerGroupId)); + + prepareAddOffsetsToTxnResponse(Errors.ABORTABLE_TRANSACTION, consumerGroupId, producerId, epoch); + runUntil(transactionManager::hasError); + assertTrue(transactionManager.lastError() instanceof AbortableTransactionException); + assertTrue(sendOffsetsResult.isCompleted()); + assertFalse(sendOffsetsResult.isSuccessful()); + assertTrue(sendOffsetsResult.error() instanceof AbortableTransactionException); + + assertAbortableError(AbortableTransactionException.class); + } + + @Test + public void testAbortableTransactionExceptionInTxnOffsetCommit() { + final TopicPartition tp = new TopicPartition("foo", 0); + + doInitTransactions(); + + transactionManager.beginTransaction(); + TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( + singletonMap(tp, new OffsetAndMetadata(39L)), new ConsumerGroupMetadata(consumerGroupId)); + + prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, producerId, epoch); + prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId); + prepareTxnOffsetCommitResponse(consumerGroupId, producerId, epoch, singletonMap(tp, Errors.ABORTABLE_TRANSACTION)); + runUntil(transactionManager::hasError); + + assertTrue(transactionManager.lastError() instanceof AbortableTransactionException); + assertTrue(sendOffsetsResult.isCompleted()); + assertFalse(sendOffsetsResult.isSuccessful()); + assertTrue(sendOffsetsResult.error() instanceof AbortableTransactionException); + assertAbortableError(AbortableTransactionException.class); + } + private FutureRecordMetadata appendToAccumulator(TopicPartition tp) throws InterruptedException { final long nowMs = time.milliseconds(); return accumulator.append(tp.topic(), tp.partition(), nowMs, "key".getBytes(), "value".getBytes(), Record.EMPTY_HEADERS, diff --git a/clients/src/test/java/org/apache/kafka/common/KafkaFutureTest.java b/clients/src/test/java/org/apache/kafka/common/KafkaFutureTest.java index f9dc5e75fc..81d2530650 100644 --- a/clients/src/test/java/org/apache/kafka/common/KafkaFutureTest.java +++ b/clients/src/test/java/org/apache/kafka/common/KafkaFutureTest.java @@ -34,6 +34,7 @@ import java.util.concurrent.TimeoutException; import java.util.function.Supplier; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -316,7 +317,7 @@ public void testThenApplyOnSucceededFutureAndFunctionThrowsCompletionException() assertIsFailed(dependantFuture); awaitAndAssertResult(future, 21, null); Throwable cause = awaitAndAssertFailure(dependantFuture, CompletionException.class, "java.lang.RuntimeException: We require more vespene gas"); - assertTrue(cause.getCause() instanceof RuntimeException); + assertInstanceOf(RuntimeException.class, cause.getCause()); assertEquals(cause.getCause().getMessage(), "We require more vespene gas"); } @@ -438,7 +439,7 @@ public void testWhenCompleteOnCancelledFuture() { assertFalse(dependantFuture.isDone()); assertTrue(future.cancel(true)); assertTrue(ran[0]); - assertTrue(err[0] instanceof CancellationException); + assertInstanceOf(CancellationException.class, err[0]); } private static class CompleterThread extends Thread { diff --git a/clients/src/test/java/org/apache/kafka/common/TopicPartitionTest.java b/clients/src/test/java/org/apache/kafka/common/TopicPartitionTest.java index fae7ce7daa..ede6918500 100644 --- a/clients/src/test/java/org/apache/kafka/common/TopicPartitionTest.java +++ b/clients/src/test/java/org/apache/kafka/common/TopicPartitionTest.java @@ -21,7 +21,7 @@ import java.io.IOException; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertEquals; /** @@ -48,7 +48,7 @@ public void testSerializationRoundtrip() throws IOException, ClassNotFoundExcept //deserialize the byteArray and check if the values are same as original Object deserializedObject = Serializer.deserialize(byteArray); - assertTrue(deserializedObject instanceof TopicPartition); + assertInstanceOf(TopicPartition.class, deserializedObject); checkValues((TopicPartition) deserializedObject); } @@ -57,7 +57,7 @@ public void testTopiPartitionSerializationCompatibility() throws IOException, Cl // assert serialized TopicPartition object in file (serializedData/topicPartitionSerializedfile) is // deserializable into TopicPartition and is compatible Object deserializedObject = Serializer.deserialize(fileName); - assertTrue(deserializedObject instanceof TopicPartition); + assertInstanceOf(TopicPartition.class, deserializedObject); checkValues((TopicPartition) deserializedObject); } } diff --git a/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java b/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java index f1d367bcc4..36c9d6f6e2 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java @@ -30,6 +30,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -40,7 +41,7 @@ public void testCreateConfigurableKafkaPrincipalBuilder() { Map configs = new HashMap<>(); configs.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, ConfigurableKafkaPrincipalBuilder.class); KafkaPrincipalBuilder builder = ChannelBuilders.createPrincipalBuilder(configs, null, null); - assertTrue(builder instanceof ConfigurableKafkaPrincipalBuilder); + assertInstanceOf(ConfigurableKafkaPrincipalBuilder.class, builder); assertTrue(((ConfigurableKafkaPrincipalBuilder) builder).configured); } 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 d92f4facb3..21ae83e16a 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 @@ -36,6 +36,7 @@ import org.apache.kafka.test.TestSslUtils; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -51,6 +52,7 @@ import java.nio.channels.Channels; import java.nio.channels.SelectionKey; import java.nio.channels.SocketChannel; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -65,13 +67,21 @@ import javax.net.ssl.SSLContext; import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLEngineResult; import javax.net.ssl.SSLParameters; +import javax.net.ssl.SSLSession; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assumptions.assumeTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** * Tests for the SSL transport layer. These use a test harness that runs a simple socket server that echos back responses. @@ -1055,7 +1065,7 @@ false, securityProtocol, config, null, null, time, new LogContext(), CertStores newServerCertStores = certBuilder(true, "server", args.useInlinePem).addHostName("localhost").build(); Map newKeystoreConfigs = newServerCertStores.keyStoreProps(); - assertTrue(serverChannelBuilder instanceof ListenerReconfigurable, "SslChannelBuilder not reconfigurable"); + assertInstanceOf(ListenerReconfigurable.class, serverChannelBuilder, "SslChannelBuilder not reconfigurable"); ListenerReconfigurable reconfigurableBuilder = (ListenerReconfigurable) serverChannelBuilder; assertEquals(listenerName, reconfigurableBuilder.listenerName()); reconfigurableBuilder.validateReconfiguration(newKeystoreConfigs); @@ -1183,7 +1193,7 @@ false, securityProtocol, config, null, null, time, new LogContext(), CertStores newClientCertStores = certBuilder(true, "client", args.useInlinePem).addHostName("localhost").build(); args.sslClientConfigs = args.getTrustingConfig(newClientCertStores, args.serverCertStores); Map newTruststoreConfigs = newClientCertStores.trustStoreProps(); - assertTrue(serverChannelBuilder instanceof ListenerReconfigurable, "SslChannelBuilder not reconfigurable"); + assertInstanceOf(ListenerReconfigurable.class, serverChannelBuilder, "SslChannelBuilder not reconfigurable"); ListenerReconfigurable reconfigurableBuilder = (ListenerReconfigurable) serverChannelBuilder; assertEquals(listenerName, reconfigurableBuilder.listenerName()); reconfigurableBuilder.validateReconfiguration(newTruststoreConfigs); @@ -1467,4 +1477,55 @@ int updateAndGet(int actualSize, boolean update) { } } } + + /** + * SSLEngine implementations may transition from NEED_UNWRAP to NEED_UNWRAP + * even after reading all the data from the socket. This test ensures we + * continue unwrapping and not break early. + * Please refer KAFKA-16305 + * for more information. + */ + @Test + public void testHandshakeUnwrapContinuesUnwrappingOnNeedUnwrapAfterAllBytesRead() throws IOException { + // Given + byte[] data = "ClientHello?".getBytes(StandardCharsets.UTF_8); + + SSLEngine sslEngine = mock(SSLEngine.class); + SocketChannel socketChannel = mock(SocketChannel.class); + SelectionKey selectionKey = mock(SelectionKey.class); + when(selectionKey.channel()).thenReturn(socketChannel); + SSLSession sslSession = mock(SSLSession.class); + SslTransportLayer sslTransportLayer = new SslTransportLayer( + "test-channel", + selectionKey, + sslEngine, + mock(ChannelMetadataRegistry.class) + ); + + when(sslEngine.getSession()).thenReturn(sslSession); + when(sslSession.getPacketBufferSize()).thenReturn(data.length * 2); + sslTransportLayer.startHandshake(); // to initialize the buffers + + ByteBuffer netReadBuffer = sslTransportLayer.netReadBuffer(); + netReadBuffer.clear(); + ByteBuffer appReadBuffer = sslTransportLayer.appReadBuffer(); + when(socketChannel.read(any(ByteBuffer.class))).then(invocation -> { + ((ByteBuffer) invocation.getArgument(0)).put(data); + return data.length; + }); + + when(sslEngine.unwrap(netReadBuffer, appReadBuffer)) + .thenAnswer(invocation -> { + netReadBuffer.flip(); + return new SSLEngineResult(SSLEngineResult.Status.OK, SSLEngineResult.HandshakeStatus.NEED_UNWRAP, data.length, 0); + }).thenReturn(new SSLEngineResult(SSLEngineResult.Status.OK, SSLEngineResult.HandshakeStatus.NEED_WRAP, 0, 0)); + + // When + SSLEngineResult result = sslTransportLayer.handshakeUnwrap(true, false); + + // Then + verify(sslEngine, times(2)).unwrap(netReadBuffer, appReadBuffer); + assertEquals(SSLEngineResult.Status.OK, result.getStatus()); + assertEquals(SSLEngineResult.HandshakeStatus.NEED_WRAP, result.getHandshakeStatus()); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java b/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java index e92ccb0519..f9d3ff3d57 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java @@ -44,6 +44,7 @@ import static org.apache.kafka.common.record.DefaultRecordBatch.RECORDS_OFFSET; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertIterableEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -425,12 +426,12 @@ public void testSkipKeyValueIteratorCorrectness(CompressionType compressionType) if (CompressionType.NONE == compressionType) { // assert that for uncompressed data stream record iterator is not used - assertTrue(skipKeyValueIterator instanceof DefaultRecordBatch.RecordIterator); + assertInstanceOf(DefaultRecordBatch.RecordIterator.class, skipKeyValueIterator); // superficial validation for correctness. Deep validation is already performed in other tests assertEquals(Utils.toList(records.records()).size(), Utils.toList(skipKeyValueIterator).size()); } else { // assert that a streaming iterator is used for compressed records - assertTrue(skipKeyValueIterator instanceof DefaultRecordBatch.StreamRecordIterator); + assertInstanceOf(DefaultRecordBatch.StreamRecordIterator.class, skipKeyValueIterator); // assert correctness for compressed records assertIterableEquals(Arrays.asList( new PartialDefaultRecord(9, (byte) 0, 0L, 1L, -1, 1, 1), 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 33a419e64e..eaaa95ff67 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 @@ -377,11 +377,8 @@ public void buildUsingLogAppendTime(Args args) { MemoryRecordsBuilder.RecordsInfo info = builder.info(); assertEquals(logAppendTime, info.maxTimestamp); - - if (args.compressionType == CompressionType.NONE && magic <= MAGIC_VALUE_V1) - assertEquals(0L, info.shallowOffsetOfMaxTimestamp); - else - assertEquals(2L, info.shallowOffsetOfMaxTimestamp); + // When logAppendTime is used, the first offset of the batch will be the offset of maxTimestamp + assertEquals(0L, info.offsetOfMaxTimestamp); for (RecordBatch batch : records.batches()) { if (magic == MAGIC_VALUE_V0) { @@ -415,10 +412,11 @@ public void buildUsingCreateTime(Args args) { assertEquals(2L, info.maxTimestamp); } - if (args.compressionType == CompressionType.NONE && magic == MAGIC_VALUE_V1) - assertEquals(1L, info.shallowOffsetOfMaxTimestamp); + if (magic == MAGIC_VALUE_V0) + // in MAGIC_VALUE_V0's case, we don't have timestamp info in records, so always return -1. + assertEquals(-1L, info.offsetOfMaxTimestamp); else - assertEquals(2L, info.shallowOffsetOfMaxTimestamp); + assertEquals(1L, info.offsetOfMaxTimestamp); int i = 0; long[] expectedTimestamps = new long[] {0L, 2L, 1L}; @@ -495,12 +493,13 @@ public void writePastLimit(Args args) { MemoryRecords records = builder.build(); MemoryRecordsBuilder.RecordsInfo info = builder.info(); - if (magic == MAGIC_VALUE_V0) + if (magic == MAGIC_VALUE_V0) { assertEquals(-1, info.maxTimestamp); - else + assertEquals(-1L, info.offsetOfMaxTimestamp); + } else { assertEquals(2L, info.maxTimestamp); - - assertEquals(2L, info.shallowOffsetOfMaxTimestamp); + assertEquals(2L, info.offsetOfMaxTimestamp); + } long i = 0L; for (RecordBatch batch : records.batches()) { 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 3f0195bf5d..9e688fc3ab 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 @@ -352,7 +352,7 @@ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { assertEquals(0, filterResult.messagesRetained()); assertEquals(DefaultRecordBatch.RECORD_BATCH_OVERHEAD, filterResult.bytesRetained()); assertEquals(12, filterResult.maxTimestamp()); - assertEquals(baseOffset + 1, filterResult.shallowOffsetOfMaxTimestamp()); + assertEquals(baseOffset + 1, filterResult.offsetOfMaxTimestamp()); // Verify filtered records filtered.flip(); @@ -413,7 +413,7 @@ protected boolean shouldRetainRecord(RecordBatch recordBatch, Record record) { assertEquals(0, filterResult.messagesRetained()); assertEquals(DefaultRecordBatch.RECORD_BATCH_OVERHEAD, filterResult.bytesRetained()); assertEquals(timestamp, filterResult.maxTimestamp()); - assertEquals(baseOffset, filterResult.shallowOffsetOfMaxTimestamp()); + assertEquals(baseOffset, filterResult.offsetOfMaxTimestamp()); assertTrue(filterResult.outputBuffer().position() > 0); // Verify filtered records @@ -893,10 +893,7 @@ public void testFilterTo(Args args) { assertEquals(filtered.limit(), result.bytesRetained()); if (magic > RecordBatch.MAGIC_VALUE_V0) { assertEquals(20L, result.maxTimestamp()); - if (compression == CompressionType.NONE && magic < RecordBatch.MAGIC_VALUE_V2) - assertEquals(4L, result.shallowOffsetOfMaxTimestamp()); - else - assertEquals(5L, result.shallowOffsetOfMaxTimestamp()); + assertEquals(4L, result.offsetOfMaxTimestamp()); } MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java index 254dea0430..5caeb7730e 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java @@ -40,6 +40,7 @@ import java.util.Collections; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -62,7 +63,7 @@ public void testSerdeUnsupportedApiVersionRequest() throws Exception { requestBuffer.flip(); RequestAndSize requestAndSize = context.parseRequest(requestBuffer); - assertTrue(requestAndSize.request instanceof ApiVersionsRequest); + assertInstanceOf(ApiVersionsRequest.class, requestAndSize.request); ApiVersionsRequest request = (ApiVersionsRequest) requestAndSize.request; assertTrue(request.hasUnsupportedRequestVersion()); diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java index db6ba89463..f80170063e 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java @@ -45,6 +45,7 @@ import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertTrue; public abstract class SaslAuthenticatorFailureDelayTest { @@ -227,7 +228,7 @@ private void createAndCheckClientAuthenticationFailure(SecurityProtocol security String mechanism, String expectedErrorMessage) throws Exception { ChannelState finalState = createAndCheckClientConnectionFailure(securityProtocol, node); Exception exception = finalState.exception(); - assertTrue(exception instanceof SaslAuthenticationException, "Invalid exception class " + exception.getClass()); + assertInstanceOf(SaslAuthenticationException.class, exception, "Invalid exception class " + exception.getClass()); if (expectedErrorMessage == null) expectedErrorMessage = "Authentication failed during authentication due to invalid credentials with SASL mechanism " + mechanism; assertEquals(expectedErrorMessage, exception.getMessage()); diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java index 98b66827e5..75684e4db3 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java @@ -140,6 +140,7 @@ import static org.apache.kafka.common.protocol.ApiKeys.LIST_OFFSETS; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -1204,7 +1205,7 @@ public void testClientLoginCallbackOverride() throws Exception { try { createClientConnection(securityProtocol, "invalid"); } catch (Exception e) { - assertTrue(e.getCause() instanceof LoginException, "Unexpected exception " + e.getCause()); + assertInstanceOf(LoginException.class, e.getCause(), "Unexpected exception " + e.getCause()); } } @@ -1805,12 +1806,12 @@ public void testValidSaslOauthBearerMechanismWithoutServerTokens() throws Except // Server with extensions, but without a token should fail to start up since it could indicate a configuration error saslServerConfigs.put("listener.name.sasl_ssl.oauthbearer." + SaslConfigs.SASL_JAAS_CONFIG, TestJaasConfig.jaasConfigProperty("OAUTHBEARER", Collections.singletonMap("unsecuredLoginExtension_test", "something"))); - try { - createEchoServer(securityProtocol); - fail("Server created with invalid login config containing extensions without a token"); - } catch (Throwable e) { - assertTrue(e.getCause() instanceof LoginException, "Unexpected exception " + Utils.stackTrace(e)); - } + + Throwable throwable = assertThrows( + Throwable.class, + () -> createEchoServer(securityProtocol), + "Server created with invalid login config containing extensions without a token"); + assertInstanceOf(LoginException.class, throwable.getCause(), "Unexpected exception " + Utils.stackTrace(throwable)); } /** @@ -2221,7 +2222,7 @@ private void createAndCheckClientAuthenticationFailure(SecurityProtocol security String mechanism, String expectedErrorMessage) throws Exception { ChannelState finalState = createAndCheckClientConnectionFailure(securityProtocol, node); Exception exception = finalState.exception(); - assertTrue(exception instanceof SaslAuthenticationException, "Invalid exception class " + exception.getClass()); + assertInstanceOf(SaslAuthenticationException.class, exception, "Invalid exception class " + exception.getClass()); String expectedExceptionMessage = expectedErrorMessage != null ? expectedErrorMessage : "Authentication failed during authentication due to invalid credentials with SASL mechanism " + mechanism; assertEquals(expectedExceptionMessage, exception.getMessage()); diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java index e7b839e4cf..c362c211e4 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginCallbackHandlerTest.java @@ -21,6 +21,7 @@ import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG; import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -213,7 +214,7 @@ public void testConfigureWithAccessTokenFile() throws Exception { Map configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString()); Map jaasConfigs = Collections.emptyMap(); configureHandler(handler, configs, jaasConfigs); - assertTrue(handler.getAccessTokenRetriever() instanceof FileTokenRetriever); + assertInstanceOf(FileTokenRetriever.class, handler.getAccessTokenRetriever()); } @Test @@ -224,7 +225,7 @@ public void testConfigureWithAccessClientCredentials() { jaasConfigs.put(CLIENT_ID_CONFIG, "an ID"); jaasConfigs.put(CLIENT_SECRET_CONFIG, "a secret"); configureHandler(handler, configs, jaasConfigs); - assertTrue(handler.getAccessTokenRetriever() instanceof HttpAccessTokenRetriever); + assertInstanceOf(HttpAccessTokenRetriever.class, handler.getAccessTokenRetriever()); } private void testInvalidAccessToken(String accessToken, String expectedMessageSubstring) throws Exception { 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 index 3b3c90bf1d..fc44297a2f 100644 --- 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 @@ -102,6 +102,18 @@ public void testRfc7688Example() throws Exception { assertEquals("143", response.extensions().map().get("port")); } + // RFC 6750 token format 1*( ALPHA / DIGIT /"-" / "." / "_" / "~" / "+" / "/" ) *"=" + @Test + public void testCharSupportForRfc6750Token() throws Exception { + String message = "n,a=user@example.com,\u0001host=server.example.com\u0001port=143\u0001" + + "auth=Bearer vF-9.df_t4qm~Tc2Nvb3RlckBhbHR+hdmlzdGEuY29/tCg==\u0001\u0001"; + OAuthBearerClientInitialResponse response = new OAuthBearerClientInitialResponse(message.getBytes(StandardCharsets.UTF_8)); + assertEquals("vF-9.df_t4qm~Tc2Nvb3RlckBhbHR+hdmlzdGEuY29/tCg==", response.tokenValue()); + assertEquals("user@example.com", response.authorizationId()); + assertEquals("server.example.com", response.extensions().map().get("host")); + assertEquals("143", response.extensions().map().get("port")); + } + @Test public void testNoExtensionsFromByteArray() throws Exception { String message = "n,a=user@example.com,\u0001" + diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java index 4cad867507..ba06650f8e 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; @@ -39,6 +40,7 @@ import java.util.concurrent.ExecutionException; import java.util.function.Consumer; import javax.security.auth.login.AppConfigurationEntry; + import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler; @@ -65,19 +67,7 @@ public abstract class OAuthBearerTest { protected void assertThrowsWithMessage(Class clazz, Executable executable, String substring) { - boolean failed = false; - - try { - executable.execute(); - } catch (Throwable t) { - failed = true; - assertTrue(clazz.isInstance(t), String.format("Test failed by exception %s, but expected %s", t.getClass(), clazz)); - - assertErrorMessageContains(t.getMessage(), substring); - } - - if (!failed) - fail("Expected test to fail with " + clazz + " that contains the string " + substring); + assertErrorMessageContains(assertThrows(clazz, executable).getMessage(), substring); } protected void assertErrorMessageContains(String actual, String expectedSubstring) { diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredValidatorCallbackHandlerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredValidatorCallbackHandlerTest.java index 0153abcc83..d7d6013a45 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredValidatorCallbackHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredValidatorCallbackHandlerTest.java @@ -17,8 +17,8 @@ package org.apache.kafka.common.security.oauthbearer.internals.unsecured; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertTrue; import java.nio.charset.StandardCharsets; import java.util.Arrays; @@ -77,8 +77,8 @@ public void validToken() { + (includeOptionalIssuedAtClaim ? comma(ISSUED_AT_CLAIM_TEXT) : "") + "}"; Object validationResult = validationResult(UNSECURED_JWT_HEADER_JSON, claimsJson, MODULE_OPTIONS_MAP_NO_SCOPE_REQUIRED); - assertTrue(validationResult instanceof OAuthBearerValidatorCallback); - assertTrue(((OAuthBearerValidatorCallback) validationResult).token() instanceof OAuthBearerUnsecuredJws); + assertInstanceOf(OAuthBearerValidatorCallback.class, validationResult); + assertInstanceOf(OAuthBearerUnsecuredJws.class, ((OAuthBearerValidatorCallback) validationResult).token()); } } @@ -103,8 +103,8 @@ public void includesRequiredScope() { String claimsJson = "{" + SUB_CLAIM_TEXT + comma(EXPIRATION_TIME_CLAIM_TEXT) + comma(SCOPE_CLAIM_TEXT) + "}"; Object validationResult = validationResult(UNSECURED_JWT_HEADER_JSON, claimsJson, MODULE_OPTIONS_MAP_REQUIRE_EXISTING_SCOPE); - assertTrue(validationResult instanceof OAuthBearerValidatorCallback); - assertTrue(((OAuthBearerValidatorCallback) validationResult).token() instanceof OAuthBearerUnsecuredJws); + assertInstanceOf(OAuthBearerValidatorCallback.class, validationResult); + assertInstanceOf(OAuthBearerUnsecuredJws.class, ((OAuthBearerValidatorCallback) validationResult).token()); } @Test @@ -123,7 +123,7 @@ private static void confirmFailsValidation(String headerJson, String claimsJson, Map moduleOptionsMap, String optionalFailureScope) throws OAuthBearerConfigException, OAuthBearerIllegalTokenException { Object validationResultObj = validationResult(headerJson, claimsJson, moduleOptionsMap); - assertTrue(validationResultObj instanceof OAuthBearerValidatorCallback); + assertInstanceOf(OAuthBearerValidatorCallback.class, validationResultObj); OAuthBearerValidatorCallback callback = (OAuthBearerValidatorCallback) validationResultObj; assertNull(callback.token()); assertNull(callback.errorOpenIDConfiguration()); diff --git a/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java index 9f02e84733..bcdb5c471a 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java @@ -51,6 +51,7 @@ import static org.apache.kafka.common.security.ssl.SslFactory.CertificateEntries.ensureCompatible; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotSame; @@ -474,7 +475,7 @@ public void testClientSpecifiedSslEngineFactoryUsed() throws Exception { clientSslConfig.put(SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG, TestSslUtils.TestSslEngineFactory.class); SslFactory sslFactory = new SslFactory(Mode.CLIENT); sslFactory.configure(clientSslConfig); - assertTrue(sslFactory.sslEngineFactory() instanceof TestSslUtils.TestSslEngineFactory, + assertInstanceOf(TestSslUtils.TestSslEngineFactory.class, sslFactory.sslEngineFactory(), "SslEngineFactory must be of expected type"); } @@ -507,7 +508,7 @@ public void testServerSpecifiedSslEngineFactoryUsed() throws Exception { serverSslConfig.put(SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG, TestSslUtils.TestSslEngineFactory.class); SslFactory sslFactory = new SslFactory(Mode.SERVER); sslFactory.configure(serverSslConfig); - assertTrue(sslFactory.sslEngineFactory() instanceof TestSslUtils.TestSslEngineFactory, + assertInstanceOf(TestSslUtils.TestSslEngineFactory.class, sslFactory.sslEngineFactory(), "SslEngineFactory must be of expected type"); } diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java index aff01e3fe8..fd9ee8395c 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java @@ -17,9 +17,9 @@ package org.apache.kafka.common.serialization; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.KafkaException; @@ -45,7 +45,7 @@ public void testListKeyDeserializerNoArgConstructorsWithClassNames() { listDeserializer.configure(props, true); final Deserializer inner = listDeserializer.innerDeserializer(); assertNotNull(inner, "Inner deserializer should be not null"); - assertTrue(inner instanceof StringDeserializer, "Inner deserializer type should be StringDeserializer"); + assertInstanceOf(StringDeserializer.class, inner, "Inner deserializer type should be StringDeserializer"); } @Test @@ -55,7 +55,7 @@ public void testListValueDeserializerNoArgConstructorsWithClassNames() { listDeserializer.configure(props, false); final Deserializer inner = listDeserializer.innerDeserializer(); assertNotNull(inner, "Inner deserializer should be not null"); - assertTrue(inner instanceof IntegerDeserializer, "Inner deserializer type should be IntegerDeserializer"); + assertInstanceOf(IntegerDeserializer.class, inner, "Inner deserializer type should be IntegerDeserializer"); } @Test @@ -65,7 +65,7 @@ public void testListKeyDeserializerNoArgConstructorsWithClassObjects() { listDeserializer.configure(props, true); final Deserializer inner = listDeserializer.innerDeserializer(); assertNotNull(inner, "Inner deserializer should be not null"); - assertTrue(inner instanceof StringDeserializer, "Inner deserializer type should be StringDeserializer"); + assertInstanceOf(StringDeserializer.class, inner, "Inner deserializer type should be StringDeserializer"); } @Test @@ -75,7 +75,7 @@ public void testListValueDeserializerNoArgConstructorsWithClassObjects() { listDeserializer.configure(props, false); final Deserializer inner = listDeserializer.innerDeserializer(); assertNotNull(inner, "Inner deserializer should be not null"); - assertTrue(inner instanceof StringDeserializer, "Inner deserializer type should be StringDeserializer"); + assertInstanceOf(StringDeserializer.class, inner, "Inner deserializer type should be StringDeserializer"); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java index a8ab191cad..94473858fe 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java @@ -17,9 +17,9 @@ package org.apache.kafka.common.serialization; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.KafkaException; @@ -43,7 +43,7 @@ public void testListKeySerializerNoArgConstructorsWithClassName() { listSerializer.configure(props, true); final Serializer inner = listSerializer.getInnerSerializer(); assertNotNull(inner, "Inner serializer should be not null"); - assertTrue(inner instanceof StringSerializer, "Inner serializer type should be StringSerializer"); + assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer"); } @Test @@ -52,7 +52,7 @@ public void testListValueSerializerNoArgConstructorsWithClassName() { listSerializer.configure(props, false); final Serializer inner = listSerializer.getInnerSerializer(); assertNotNull(inner, "Inner serializer should be not null"); - assertTrue(inner instanceof StringSerializer, "Inner serializer type should be StringSerializer"); + assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer"); } @Test @@ -61,7 +61,7 @@ public void testListKeySerializerNoArgConstructorsWithClassObject() { listSerializer.configure(props, true); final Serializer inner = listSerializer.getInnerSerializer(); assertNotNull(inner, "Inner serializer should be not null"); - assertTrue(inner instanceof StringSerializer, "Inner serializer type should be StringSerializer"); + assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer"); } @Test @@ -70,7 +70,7 @@ public void testListValueSerializerNoArgConstructorsWithClassObject() { listSerializer.configure(props, false); final Serializer inner = listSerializer.getInnerSerializer(); assertNotNull(inner, "Inner serializer should be not null"); - assertTrue(inner instanceof StringSerializer, "Inner serializer type should be StringSerializer"); + assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer"); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java b/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java index 12672a30f6..9652ebfd37 100644 --- a/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java +++ b/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java @@ -59,6 +59,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -247,7 +248,7 @@ public void testCreateRequestSubscriptionNeeded() { Optional> requestOptional = telemetrySender.createRequest(); assertNotNull(requestOptional); assertTrue(requestOptional.isPresent()); - assertTrue(requestOptional.get().build() instanceof GetTelemetrySubscriptionsRequest); + assertInstanceOf(GetTelemetrySubscriptionsRequest.class, requestOptional.get().build()); GetTelemetrySubscriptionsRequest request = (GetTelemetrySubscriptionsRequest) requestOptional.get().build(); GetTelemetrySubscriptionsRequest expectedResult = new GetTelemetrySubscriptionsRequest.Builder( @@ -266,7 +267,7 @@ public void testCreateRequestSubscriptionNeededAfterExistingSubscription() { Optional> requestOptional = telemetrySender.createRequest(); assertNotNull(requestOptional); assertTrue(requestOptional.isPresent()); - assertTrue(requestOptional.get().build() instanceof GetTelemetrySubscriptionsRequest); + assertInstanceOf(GetTelemetrySubscriptionsRequest.class, requestOptional.get().build()); GetTelemetrySubscriptionsRequest request = (GetTelemetrySubscriptionsRequest) requestOptional.get().build(); GetTelemetrySubscriptionsRequest expectedResult = new GetTelemetrySubscriptionsRequest.Builder( @@ -290,7 +291,7 @@ public void testCreateRequestPushNeeded() { Optional> requestOptional = telemetrySender.createRequest(); assertNotNull(requestOptional); assertTrue(requestOptional.isPresent()); - assertTrue(requestOptional.get().build() instanceof PushTelemetryRequest); + assertInstanceOf(PushTelemetryRequest.class, requestOptional.get().build()); PushTelemetryRequest request = (PushTelemetryRequest) requestOptional.get().build(); PushTelemetryRequest expectedResult = new PushTelemetryRequest.Builder( @@ -373,7 +374,7 @@ public void testCreateRequestPushCompression(CompressionType compressionType) { Optional> requestOptional = telemetrySender.createRequest(); assertNotNull(requestOptional); assertTrue(requestOptional.isPresent()); - assertTrue(requestOptional.get().build() instanceof PushTelemetryRequest); + assertInstanceOf(PushTelemetryRequest.class, requestOptional.get().build()); PushTelemetryRequest request = (PushTelemetryRequest) requestOptional.get().build(); assertEquals(subscription.clientInstanceId(), request.data().clientInstanceId()); @@ -401,7 +402,7 @@ public void testCreateRequestPushCompressionException() { Optional> requestOptional = telemetrySender.createRequest(); assertNotNull(requestOptional); assertTrue(requestOptional.isPresent()); - assertTrue(requestOptional.get().build() instanceof PushTelemetryRequest); + assertInstanceOf(PushTelemetryRequest.class, requestOptional.get().build()); PushTelemetryRequest request = (PushTelemetryRequest) requestOptional.get().build(); assertEquals(subscription.clientInstanceId(), request.data().clientInstanceId()); diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java index db86558f7d..ec4cc412d8 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java @@ -37,6 +37,7 @@ import org.apache.kafka.common.requests.ByteBufferChannel; import org.apache.kafka.common.requests.MetadataResponse.PartitionMetadata; import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -73,6 +74,7 @@ import static java.util.Arrays.asList; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -234,6 +236,22 @@ public static File tempDirectory() { return tempDirectory(null); } + /** + * Create a temporary directory under the given root directory. + * The root directory is removed on JVM exit if it doesn't already exist + * when this function is invoked. + * + * @param root path to create temporary directory under + * @return the temporary directory created within {@code root} + */ + public static File tempRelativeDir(String root) { + File rootFile = new File(root); + if (rootFile.mkdir()) { + rootFile.deleteOnExit(); + } + return tempDirectory(rootFile.toPath(), null); + } + /** * Create a temporary relative directory in the specified parent directory with the given prefix. * @@ -279,6 +297,14 @@ public static Properties producerConfig(final String bootstrapServers, final Cla return producerConfig(bootstrapServers, keySerializer, valueSerializer, new Properties()); } + public static Properties requiredConsumerConfig() { + final Properties consumerConfig = new Properties(); + consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091"); + consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + return consumerConfig; + } + public static Properties consumerConfig(final String bootstrapServers, final String groupId, final Class keyDeserializer, @@ -531,7 +557,7 @@ public static Set generateRandomTopicPartitions(int numTopic, in */ public static T assertFutureThrows(Future future, Class exceptionCauseClass) { ExecutionException exception = assertThrows(ExecutionException.class, future::get); - assertTrue(exceptionCauseClass.isInstance(exception.getCause()), + assertInstanceOf(exceptionCauseClass, exception.getCause(), "Unexpected exception cause " + exception.getCause()); return exceptionCauseClass.cast(exception.getCause()); } 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 cbaeeae66f..7b78c64af0 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 @@ -424,7 +424,7 @@ protected static Object convertTo(Schema toSchema, Schema fromSchema, Object val return BigDecimal.valueOf(converted); } if (value instanceof String) { - return new BigDecimal(value.toString()).doubleValue(); + return new BigDecimal(value.toString()); } } if (value instanceof ByteBuffer) { @@ -802,11 +802,12 @@ protected static SchemaAndValue parse(Parser parser, boolean embedded) throws No try { if (parser.canConsume(ARRAY_BEGIN_DELIMITER)) { List result = new ArrayList<>(); + boolean compatible = true; Schema elementSchema = null; while (parser.hasNext()) { if (parser.canConsume(ARRAY_END_DELIMITER)) { Schema listSchema; - if (elementSchema != null) { + if (elementSchema != null && compatible) { listSchema = SchemaBuilder.array(elementSchema).schema(); result = alignListEntriesWithSchema(listSchema, result); } else { @@ -821,6 +822,9 @@ protected static SchemaAndValue parse(Parser parser, boolean embedded) throws No } SchemaAndValue element = parse(parser, true); elementSchema = commonSchemaFor(elementSchema, element); + if (elementSchema == null && element != null && element.schema() != null) { + compatible = false; + } result.add(element != null ? element.value() : null); int currentPosition = parser.mark(); @@ -840,15 +844,17 @@ protected static SchemaAndValue parse(Parser parser, boolean embedded) throws No if (parser.canConsume(MAP_BEGIN_DELIMITER)) { Map result = new LinkedHashMap<>(); + boolean keyCompatible = true; Schema keySchema = null; + boolean valueCompatible = true; Schema valueSchema = null; while (parser.hasNext()) { if (parser.canConsume(MAP_END_DELIMITER)) { Schema mapSchema; - if (keySchema != null && valueSchema != null) { + if (keySchema != null && valueSchema != null && keyCompatible && valueCompatible) { mapSchema = SchemaBuilder.map(keySchema, valueSchema).build(); result = alignMapKeysAndValuesWithSchema(mapSchema, result); - } else if (keySchema != null) { + } else if (keySchema != null && keyCompatible) { mapSchema = SchemaBuilder.mapWithNullValues(keySchema); result = alignMapKeysWithSchema(mapSchema, result); } else { @@ -876,7 +882,13 @@ protected static SchemaAndValue parse(Parser parser, boolean embedded) throws No parser.canConsume(COMMA_DELIMITER); keySchema = commonSchemaFor(keySchema, key); + if (keySchema == null && key.schema() != null) { + keyCompatible = false; + } valueSchema = commonSchemaFor(valueSchema, value); + if (valueSchema == null && value != null && value.schema() != null) { + valueCompatible = false; + } } // Missing either a comma or an end delimiter if (COMMA_DELIMITER.equals(parser.previous())) { diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java index 3700a6ee4e..3aad588fc2 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java @@ -25,6 +25,7 @@ import java.math.BigDecimal; import java.math.BigInteger; +import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.text.SimpleDateFormat; import java.util.ArrayList; @@ -37,6 +38,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -363,33 +365,73 @@ public void shouldConvertStringOfListWithOnlyNumericElementTypesIntoListOfLarges } /** - * The parsed array has byte values and one int value, so we should return list with single unified type of integers. + * We parse into different element types, but cannot infer a common element schema. + * This behavior should be independent of the order that the elements appear in the string */ @Test - public void shouldConvertStringOfListWithMixedElementTypesIntoListWithDifferentElementTypes() { - String str = "[1, 2, \"three\"]"; - List list = Values.convertToList(Schema.STRING_SCHEMA, str); - assertEquals(3, list.size()); - assertEquals(1, ((Number) list.get(0)).intValue()); - assertEquals(2, ((Number) list.get(1)).intValue()); - assertEquals("three", list.get(2)); + public void shouldParseStringListWithMultipleElementTypes() { + assertParseStringArrayWithNoSchema( + Arrays.asList((byte) 1, (byte) 2, (short) 300, "four"), + "[1, 2, 300, \"four\"]"); + assertParseStringArrayWithNoSchema( + Arrays.asList((byte) 2, (short) 300, "four", (byte) 1), + "[2, 300, \"four\", 1]"); + assertParseStringArrayWithNoSchema( + Arrays.asList((short) 300, "four", (byte) 1, (byte) 2), + "[300, \"four\", 1, 2]"); + assertParseStringArrayWithNoSchema( + Arrays.asList("four", (byte) 1, (byte) 2, (short) 300), + "[\"four\", 1, 2, 300]"); + } + + private void assertParseStringArrayWithNoSchema(List expected, String str) { + SchemaAndValue result = Values.parseString(str); + assertEquals(Type.ARRAY, result.schema().type()); + assertNull(result.schema().valueSchema()); + List list = (List) result.value(); + assertEquals(expected, list); } /** - * We parse into different element types, but cannot infer a common element schema. + * Maps with an inconsistent key type don't find a common type for the keys or the values + * This behavior should be independent of the order that the pairs appear in the string + */ + @Test + public void shouldParseStringMapWithMultipleKeyTypes() { + Map expected = new HashMap<>(); + expected.put((byte) 1, (byte) 1); + expected.put((byte) 2, (byte) 1); + expected.put((short) 300, (short) 300); + expected.put("four", (byte) 1); + assertParseStringMapWithNoSchema(expected, "{1:1, 2:1, 300:300, \"four\":1}"); + assertParseStringMapWithNoSchema(expected, "{2:1, 300:300, \"four\":1, 1:1}"); + assertParseStringMapWithNoSchema(expected, "{300:300, \"four\":1, 1:1, 2:1}"); + assertParseStringMapWithNoSchema(expected, "{\"four\":1, 1:1, 2:1, 300:300}"); + } + + /** + * Maps with a consistent key type may still not have a common type for the values + * This behavior should be independent of the order that the pairs appear in the string */ @Test - public void shouldParseStringListWithMultipleElementTypesAndReturnListWithNoSchema() { - String str = "[1, 2, 3, \"four\"]"; + public void shouldParseStringMapWithMultipleValueTypes() { + Map expected = new HashMap<>(); + expected.put((short) 1, (byte) 1); + expected.put((short) 2, (byte) 1); + expected.put((short) 300, (short) 300); + expected.put((short) 4, "four"); + assertParseStringMapWithNoSchema(expected, "{1:1, 2:1, 300:300, 4:\"four\"}"); + assertParseStringMapWithNoSchema(expected, "{2:1, 300:300, 4:\"four\", 1:1}"); + assertParseStringMapWithNoSchema(expected, "{300:300, 4:\"four\", 1:1, 2:1}"); + assertParseStringMapWithNoSchema(expected, "{4:\"four\", 1:1, 2:1, 300:300}"); + } + + private void assertParseStringMapWithNoSchema(Map expected, String str) { SchemaAndValue result = Values.parseString(str); - assertEquals(Type.ARRAY, result.schema().type()); + assertEquals(Type.MAP, result.schema().type()); assertNull(result.schema().valueSchema()); - List list = (List) result.value(); - assertEquals(4, list.size()); - assertEquals(1, ((Number) list.get(0)).intValue()); - assertEquals(2, ((Number) list.get(1)).intValue()); - assertEquals(3, ((Number) list.get(2)).intValue()); - assertEquals("four", list.get(3)); + Map list = (Map) result.value(); + assertEquals(expected, list); } /** @@ -744,6 +786,39 @@ public void shouldConvertTimestampValues() { assertEquals(current, ts4); } + @Test + public void shouldConvertDecimalValues() { + // Various forms of the same number should all be parsed to the same BigDecimal + Number number = 1.0f; + String string = number.toString(); + BigDecimal value = new BigDecimal(string); + byte[] bytes = Decimal.fromLogical(Decimal.schema(1), value); + ByteBuffer buffer = ByteBuffer.wrap(bytes); + + assertEquals(value, Values.convertToDecimal(null, number, 1)); + assertEquals(value, Values.convertToDecimal(null, string, 1)); + assertEquals(value, Values.convertToDecimal(null, value, 1)); + assertEquals(value, Values.convertToDecimal(null, bytes, 1)); + assertEquals(value, Values.convertToDecimal(null, buffer, 1)); + } + + /** + * Test parsing distinct number-like types (strings containing numbers, and logical Decimals) in the same list + * The parser does not convert Numbers to Decimals, or Strings containing numbers to Numbers automatically. + */ + @Test + public void shouldNotConvertArrayValuesToDecimal() { + List decimals = Arrays.asList("\"1.0\"", BigDecimal.valueOf(Long.MAX_VALUE).add(BigDecimal.ONE), + BigDecimal.valueOf(Long.MIN_VALUE).subtract(BigDecimal.ONE), (byte) 1, (byte) 1); + List expected = new ArrayList<>(decimals); // most values are directly reproduced with the same type + expected.set(0, "1.0"); // The quotes are parsed away, but the value remains a string + SchemaAndValue schemaAndValue = Values.parseString(decimals.toString()); + Schema schema = schemaAndValue.schema(); + assertEquals(Type.ARRAY, schema.type()); + assertNull(schema.valueSchema()); + assertEquals(expected, schemaAndValue.value()); + } + @Test public void canConsume() { } @@ -755,14 +830,14 @@ public void shouldParseBigIntegerAsDecimalWithZeroScale() { String.valueOf(value) ); assertEquals(Decimal.schema(0), schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof BigDecimal); + assertInstanceOf(BigDecimal.class, schemaAndValue.value()); assertEquals(value, ((BigDecimal) schemaAndValue.value()).unscaledValue()); value = BigInteger.valueOf(Long.MIN_VALUE).subtract(new BigInteger("1")); schemaAndValue = Values.parseString( String.valueOf(value) ); assertEquals(Decimal.schema(0), schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof BigDecimal); + assertInstanceOf(BigDecimal.class, schemaAndValue.value()); assertEquals(value, ((BigDecimal) schemaAndValue.value()).unscaledValue()); } @@ -773,14 +848,14 @@ public void shouldParseByteAsInt8() { String.valueOf(value) ); assertEquals(Schema.INT8_SCHEMA, schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof Byte); + assertInstanceOf(Byte.class, schemaAndValue.value()); assertEquals(value.byteValue(), ((Byte) schemaAndValue.value()).byteValue()); value = Byte.MIN_VALUE; schemaAndValue = Values.parseString( String.valueOf(value) ); assertEquals(Schema.INT8_SCHEMA, schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof Byte); + assertInstanceOf(Byte.class, schemaAndValue.value()); assertEquals(value.byteValue(), ((Byte) schemaAndValue.value()).byteValue()); } @@ -791,14 +866,14 @@ public void shouldParseShortAsInt16() { String.valueOf(value) ); assertEquals(Schema.INT16_SCHEMA, schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof Short); + assertInstanceOf(Short.class, schemaAndValue.value()); assertEquals(value.shortValue(), ((Short) schemaAndValue.value()).shortValue()); value = Short.MIN_VALUE; schemaAndValue = Values.parseString( String.valueOf(value) ); assertEquals(Schema.INT16_SCHEMA, schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof Short); + assertInstanceOf(Short.class, schemaAndValue.value()); assertEquals(value.shortValue(), ((Short) schemaAndValue.value()).shortValue()); } @@ -809,14 +884,14 @@ public void shouldParseIntegerAsInt32() { String.valueOf(value) ); assertEquals(Schema.INT32_SCHEMA, schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof Integer); + assertInstanceOf(Integer.class, schemaAndValue.value()); assertEquals(value.intValue(), ((Integer) schemaAndValue.value()).intValue()); value = Integer.MIN_VALUE; schemaAndValue = Values.parseString( String.valueOf(value) ); assertEquals(Schema.INT32_SCHEMA, schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof Integer); + assertInstanceOf(Integer.class, schemaAndValue.value()); assertEquals(value.intValue(), ((Integer) schemaAndValue.value()).intValue()); } @@ -827,14 +902,14 @@ public void shouldParseLongAsInt64() { String.valueOf(value) ); assertEquals(Schema.INT64_SCHEMA, schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof Long); + assertInstanceOf(Long.class, schemaAndValue.value()); assertEquals(value.longValue(), ((Long) schemaAndValue.value()).longValue()); value = Long.MIN_VALUE; schemaAndValue = Values.parseString( String.valueOf(value) ); assertEquals(Schema.INT64_SCHEMA, schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof Long); + assertInstanceOf(Long.class, schemaAndValue.value()); assertEquals(value.longValue(), ((Long) schemaAndValue.value()).longValue()); } @@ -845,14 +920,14 @@ public void shouldParseFloatAsFloat32() { String.valueOf(value) ); assertEquals(Schema.FLOAT32_SCHEMA, schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof Float); + assertInstanceOf(Float.class, schemaAndValue.value()); assertEquals(value, (Float) schemaAndValue.value(), 0); value = -Float.MAX_VALUE; schemaAndValue = Values.parseString( String.valueOf(value) ); assertEquals(Schema.FLOAT32_SCHEMA, schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof Float); + assertInstanceOf(Float.class, schemaAndValue.value()); assertEquals(value, (Float) schemaAndValue.value(), 0); } @@ -863,14 +938,14 @@ public void shouldParseDoubleAsFloat64() { String.valueOf(value) ); assertEquals(Schema.FLOAT64_SCHEMA, schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof Double); + assertInstanceOf(Double.class, schemaAndValue.value()); assertEquals(value, (Double) schemaAndValue.value(), 0); value = -Double.MAX_VALUE; schemaAndValue = Values.parseString( String.valueOf(value) ); assertEquals(Schema.FLOAT64_SCHEMA, schemaAndValue.schema()); - assertTrue(schemaAndValue.value() instanceof Double); + assertInstanceOf(Double.class, schemaAndValue.value()); assertEquals(value, (Double) schemaAndValue.value(), 0); } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorUtilsTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorUtilsTest.java index a73997ad57..99cdb61075 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorUtilsTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorUtilsTest.java @@ -32,8 +32,8 @@ import java.util.Map; import java.util.concurrent.ExecutionException; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; @@ -121,7 +121,7 @@ public void testCreateCompactedTopicFailsWithInvalidConfigurationException() thr when(admin.createTopics(any(), any())).thenReturn(ctr); Throwable ce = assertThrows(ConnectException.class, () -> MirrorUtils.createCompactedTopic(TOPIC, (short) 1, (short) 1, admin), "Should have exception thrown"); - assertTrue(ce.getCause() instanceof InvalidConfigurationException); + assertInstanceOf(InvalidConfigurationException.class, ce.getCause()); verify(future).get(); verify(ctr).values(); verify(admin).createTopics(any(), any()); @@ -135,7 +135,7 @@ public void testCreateCompactedTopicFailsWithTimeoutException() throws Exception when(admin.createTopics(any(), any())).thenReturn(ctr); Throwable ce = assertThrows(ConnectException.class, () -> MirrorUtils.createCompactedTopic(TOPIC, (short) 1, (short) 1, admin), "Should have exception thrown"); - assertTrue(ce.getCause() instanceof TimeoutException); + assertInstanceOf(TimeoutException.class, ce.getCause()); verify(future).get(); verify(ctr).values(); verify(admin).createTopics(any(), any()); 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 78a61e755f..25d8b54d4d 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 @@ -354,6 +354,11 @@ Map lastCommittedOffsets() { return Collections.unmodifiableMap(lastCommittedOffsets); } + //VisibleForTesting + Map currentOffsets() { + return Collections.unmodifiableMap(currentOffsets); + } + private void doCommitSync(Map offsets, int seqno) { log.debug("{} Committing offsets synchronously using sequence number {}: {}", this, seqno, offsets); try { 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 8558009ee9..9a47a0e753 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 @@ -61,7 +61,7 @@ public RestClient(AbstractConfig config) { } // VisibleForTesting - HttpClient httpClient(SslContextFactory sslContextFactory) { + HttpClient httpClient(SslContextFactory.Client sslContextFactory) { return sslContextFactory != null ? new HttpClient(sslContextFactory) : new HttpClient(); } @@ -120,7 +120,7 @@ public HttpResponse httpRequest(String url, String method, HttpHeaders he Objects.requireNonNull(method, "method must be non-null"); Objects.requireNonNull(responseFormat, "response format must be non-null"); // Only try to load SSL configs if we have to (see KAFKA-14816) - SslContextFactory sslContextFactory = url.startsWith("https://") + SslContextFactory.Client sslContextFactory = url.startsWith("https://") ? SSLUtils.createClientSideSslContextFactory(config) : null; HttpClient client = httpClient(sslContextFactory); 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 f078b24420..b6bb4fa3bd 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 @@ -159,7 +159,7 @@ public final Connector createConnector(String listener, boolean isAdmin) { ServerConnector connector; if (PROTOCOL_HTTPS.equals(protocol)) { - SslContextFactory ssl; + SslContextFactory.Server ssl; if (isAdmin) { ssl = SSLUtils.createServerSideSslContextFactory(config, RestServerConfig.ADMIN_LISTENERS_HTTPS_CONFIGS_PREFIX); } else { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java index efb184849f..341e3f3550 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java @@ -40,7 +40,7 @@ public class SSLUtils { /** * Configures SSL/TLS for HTTPS Jetty Server using configs with the given prefix */ - public static SslContextFactory createServerSideSslContextFactory(AbstractConfig config, String prefix) { + public static SslContextFactory.Server createServerSideSslContextFactory(AbstractConfig config, String prefix) { Map sslConfigValues = config.valuesWithPrefixAllOrNothing(prefix); final SslContextFactory.Server ssl = new SslContextFactory.Server(); @@ -56,14 +56,14 @@ public static SslContextFactory createServerSideSslContextFactory(AbstractConfig /** * Configures SSL/TLS for HTTPS Jetty Server */ - public static SslContextFactory createServerSideSslContextFactory(AbstractConfig config) { + public static SslContextFactory.Server createServerSideSslContextFactory(AbstractConfig config) { return createServerSideSslContextFactory(config, "listeners.https."); } /** * Configures SSL/TLS for HTTPS Jetty Client */ - public static SslContextFactory createClientSideSslContextFactory(AbstractConfig config) { + public static SslContextFactory.Client createClientSideSslContextFactory(AbstractConfig config) { Map sslConfigValues = config.valuesWithPrefixAllOrNothing("listeners.https."); final SslContextFactory.Client ssl = new SslContextFactory.Client(); @@ -147,7 +147,7 @@ protected static void configureSslContextFactoryAlgorithms(SslContextFactory ssl /** * Configures hostname verification related settings in SslContextFactory */ - protected static void configureSslContextFactoryEndpointIdentification(SslContextFactory ssl, Map sslConfigValues) { + protected static void configureSslContextFactoryEndpointIdentification(SslContextFactory.Client ssl, Map sslConfigValues) { String sslEndpointIdentificationAlg = (String) sslConfigValues.get(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG); if (sslEndpointIdentificationAlg != null) ssl.setEndpointIdentificationAlgorithm(sslEndpointIdentificationAlg); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetUtils.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetUtils.java index 1d6632e60c..6c98f42792 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetUtils.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetUtils.java @@ -115,8 +115,10 @@ public static void processPartitionKey(byte[] partitionKey, byte[] offsetValue, } if (!(keyList.get(1) instanceof Map)) { - log.warn("Ignoring offset partition key with an unexpected format for the second element in the partition key list. " + - "Expected type: {}, actual type: {}", Map.class.getName(), className(keyList.get(1))); + if (keyList.get(1) != null) { + log.warn("Ignoring offset partition key with an unexpected format for the second element in the partition key list. " + + "Expected type: {}, actual type: {}", Map.class.getName(), className(keyList.get(1))); + } return; } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java index 543703a13a..2c7250eb58 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java @@ -53,22 +53,22 @@ private DocInfo(String transformationName, String overview, ConfigDef configDef) } private static final List TRANSFORMATIONS = Arrays.asList( + new DocInfo(Cast.class.getName(), Cast.OVERVIEW_DOC, Cast.CONFIG_DEF), + new DocInfo(DropHeaders.class.getName(), DropHeaders.OVERVIEW_DOC, DropHeaders.CONFIG_DEF), + new DocInfo(ExtractField.class.getName(), ExtractField.OVERVIEW_DOC, ExtractField.CONFIG_DEF), + new DocInfo(Filter.class.getName(), Filter.OVERVIEW_DOC, Filter.CONFIG_DEF), + new DocInfo(Flatten.class.getName(), Flatten.OVERVIEW_DOC, Flatten.CONFIG_DEF), + new DocInfo(HeaderFrom.class.getName(), HeaderFrom.OVERVIEW_DOC, HeaderFrom.CONFIG_DEF), + new DocInfo(HoistField.class.getName(), HoistField.OVERVIEW_DOC, HoistField.CONFIG_DEF), new DocInfo(InsertField.class.getName(), InsertField.OVERVIEW_DOC, InsertField.CONFIG_DEF), - new DocInfo(ReplaceField.class.getName(), ReplaceField.OVERVIEW_DOC, ReplaceField.CONFIG_DEF), + new DocInfo(InsertHeader.class.getName(), InsertHeader.OVERVIEW_DOC, InsertHeader.CONFIG_DEF), new DocInfo(MaskField.class.getName(), MaskField.OVERVIEW_DOC, MaskField.CONFIG_DEF), - new DocInfo(ValueToKey.class.getName(), ValueToKey.OVERVIEW_DOC, ValueToKey.CONFIG_DEF), - new DocInfo(HoistField.class.getName(), HoistField.OVERVIEW_DOC, HoistField.CONFIG_DEF), - new DocInfo(ExtractField.class.getName(), ExtractField.OVERVIEW_DOC, ExtractField.CONFIG_DEF), - new DocInfo(SetSchemaMetadata.class.getName(), SetSchemaMetadata.OVERVIEW_DOC, SetSchemaMetadata.CONFIG_DEF), - new DocInfo(TimestampRouter.class.getName(), TimestampRouter.OVERVIEW_DOC, TimestampRouter.CONFIG_DEF), new DocInfo(RegexRouter.class.getName(), RegexRouter.OVERVIEW_DOC, RegexRouter.CONFIG_DEF), - new DocInfo(Flatten.class.getName(), Flatten.OVERVIEW_DOC, Flatten.CONFIG_DEF), - new DocInfo(Cast.class.getName(), Cast.OVERVIEW_DOC, Cast.CONFIG_DEF), + new DocInfo(ReplaceField.class.getName(), ReplaceField.OVERVIEW_DOC, ReplaceField.CONFIG_DEF), + new DocInfo(SetSchemaMetadata.class.getName(), SetSchemaMetadata.OVERVIEW_DOC, SetSchemaMetadata.CONFIG_DEF), new DocInfo(TimestampConverter.class.getName(), TimestampConverter.OVERVIEW_DOC, TimestampConverter.CONFIG_DEF), - new DocInfo(Filter.class.getName(), Filter.OVERVIEW_DOC, Filter.CONFIG_DEF), - new DocInfo(InsertHeader.class.getName(), InsertHeader.OVERVIEW_DOC, InsertHeader.CONFIG_DEF), - new DocInfo(DropHeaders.class.getName(), DropHeaders.OVERVIEW_DOC, DropHeaders.CONFIG_DEF), - new DocInfo(HeaderFrom.class.getName(), HeaderFrom.OVERVIEW_DOC, HeaderFrom.CONFIG_DEF) + new DocInfo(TimestampRouter.class.getName(), TimestampRouter.OVERVIEW_DOC, TimestampRouter.CONFIG_DEF), + new DocInfo(ValueToKey.class.getName(), ValueToKey.OVERVIEW_DOC, ValueToKey.CONFIG_DEF) ); private static void printTransformationHtml(PrintStream out, DocInfo docInfo) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java index d651ef8709..35bfeab7c3 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java @@ -78,6 +78,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; /** * Test simple operations on the workers of a Connect cluster. @@ -860,7 +861,7 @@ private void assertTimeoutException(Runnable operation, String expectedStageDesc return false; } catch (Throwable t) { latestError.set(t); - assertTrue(t instanceof ConnectRestException); + assertInstanceOf(ConnectRestException.class, t); ConnectRestException restException = (ConnectRestException) t; assertEquals(INTERNAL_SERVER_ERROR.getStatusCode(), restException.statusCode()); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java index 09ea3f5ae4..04dd4c7de6 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java @@ -104,6 +104,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; @Category(IntegrationTest.class) public class ExactlyOnceSourceIntegrationTest { @@ -1078,7 +1079,7 @@ private Map> parseOffsetForTasks(ConsumerRecords T assertAndCast(Object o, Class klass, String objectDescription) { String className = o == null ? "null" : o.getClass().getName(); - assertTrue(objectDescription + " should be " + klass.getName() + "; was " + className + " instead", klass.isInstance(o)); + assertInstanceOf(klass, o, objectDescription + " should be " + klass.getName() + "; was " + className + " instead"); return (T) o; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RestForwardingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RestForwardingIntegrationTest.java index 0bbad10a57..2e63a2518b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RestForwardingIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RestForwardingIntegrationTest.java @@ -89,7 +89,7 @@ public class RestForwardingIntegrationTest { @Mock private Herder leaderHerder; - private SslContextFactory factory; + private SslContextFactory.Client factory; private CloseableHttpClient httpClient; private Collection responses; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java index 9ced632b90..f91e37ef9f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java @@ -51,6 +51,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.isNull; @@ -529,7 +530,7 @@ public void testFailConnectorThatIsNeitherSourceNorSink() { ArgumentCaptor exceptionCapture = ArgumentCaptor.forClass(Throwable.class); verify(listener).onFailure(eq(CONNECTOR), exceptionCapture.capture()); Throwable e = exceptionCapture.getValue(); - assertTrue(e instanceof ConnectException); + assertInstanceOf(ConnectException.class, e); assertTrue(e.getMessage().contains("must be a subclass of")); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskMockitoTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskMockitoTest.java deleted file mode 100644 index f39dce6646..0000000000 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskMockitoTest.java +++ /dev/null @@ -1,1084 +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.connect.runtime; - -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -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.consumer.MockConsumer; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.consumer.OffsetCommitCallback; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; -import org.apache.kafka.common.MetricName; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.WakeupException; -import org.apache.kafka.common.header.Header; -import org.apache.kafka.common.header.Headers; -import org.apache.kafka.common.header.internals.RecordHeaders; -import org.apache.kafka.common.record.RecordBatch; -import org.apache.kafka.common.record.TimestampType; -import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaAndValue; -import org.apache.kafka.connect.errors.RetriableException; -import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; -import org.apache.kafka.connect.runtime.WorkerSinkTask.SinkTaskMetricsGroup; -import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics; -import org.apache.kafka.connect.runtime.errors.ErrorReporter; -import org.apache.kafka.connect.runtime.errors.ProcessingContext; -import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; -import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest; -import org.apache.kafka.connect.runtime.isolation.PluginClassLoader; -import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; -import org.apache.kafka.connect.sink.SinkConnector; -import org.apache.kafka.connect.sink.SinkRecord; -import org.apache.kafka.connect.sink.SinkTask; -import org.apache.kafka.connect.storage.ClusterConfigState; -import org.apache.kafka.connect.storage.Converter; -import org.apache.kafka.connect.storage.HeaderConverter; -import org.apache.kafka.connect.storage.StatusBackingStore; -import org.apache.kafka.connect.storage.StringConverter; -import org.apache.kafka.connect.util.ConnectorTaskId; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnit; -import org.mockito.junit.MockitoJUnitRunner; -import org.mockito.junit.MockitoRule; -import org.mockito.quality.Strictness; -import org.mockito.stubbing.Answer; -import org.mockito.stubbing.OngoingStubbing; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.function.Supplier; -import java.util.regex.Pattern; - -import static java.util.Arrays.asList; -import static java.util.Collections.singleton; -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; -import static org.junit.Assert.fail; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyList; -import static org.mockito.ArgumentMatchers.anyMap; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.atLeastOnce; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -@RunWith(MockitoJUnitRunner.StrictStubs.class) -public class WorkerSinkTaskMockitoTest { - // These are fixed to keep this code simpler. In this example we assume byte[] raw values - // with mix of integer/string in Connect - private static final String TOPIC = "test"; - private static final int PARTITION = 12; - private static final int PARTITION2 = 13; - private static final int PARTITION3 = 14; - private static final long FIRST_OFFSET = 45; - private static final Schema KEY_SCHEMA = Schema.INT32_SCHEMA; - private static final int KEY = 12; - private static final Schema VALUE_SCHEMA = Schema.STRING_SCHEMA; - private static final String VALUE = "VALUE"; - private static final byte[] RAW_KEY = "key".getBytes(); - private static final byte[] RAW_VALUE = "value".getBytes(); - - private static final TopicPartition TOPIC_PARTITION = new TopicPartition(TOPIC, PARTITION); - private static final TopicPartition TOPIC_PARTITION2 = new TopicPartition(TOPIC, PARTITION2); - private static final TopicPartition TOPIC_PARTITION3 = new TopicPartition(TOPIC, PARTITION3); - - private static final Set INITIAL_ASSIGNMENT = - new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2)); - - private static final Map TASK_PROPS = new HashMap<>(); - - static { - TASK_PROPS.put(SinkConnector.TOPICS_CONFIG, TOPIC); - TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, SinkTask.class.getName()); - } - - private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); - - private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); - private ConnectorTaskId taskId1 = new ConnectorTaskId("job", 1); - private TargetState initialState = TargetState.STARTED; - private MockTime time; - private WorkerSinkTask workerTask; - @Mock - private SinkTask sinkTask; - private ArgumentCaptor sinkTaskContext = ArgumentCaptor.forClass(WorkerSinkTaskContext.class); - private WorkerConfig workerConfig; - private MockConnectMetrics metrics; - @Mock - private PluginClassLoader pluginLoader; - @Mock - private Converter keyConverter; - @Mock - private Converter valueConverter; - @Mock - private HeaderConverter headerConverter; - @Mock - private TransformationChain, SinkRecord> transformationChain; - @Mock - private TaskStatus.Listener statusListener; - @Mock - private StatusBackingStore statusBackingStore; - @Mock - private KafkaConsumer consumer; - @Mock - private ErrorHandlingMetrics errorHandlingMetrics; - private ArgumentCaptor rebalanceListener = ArgumentCaptor.forClass(ConsumerRebalanceListener.class); - @Rule - public final MockitoRule rule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); - - private long recordsReturnedTp1; - private long recordsReturnedTp3; - - @Before - public void setUp() { - time = new MockTime(); - Map workerProps = new HashMap<>(); - workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); - workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); - workerConfig = new StandaloneConfig(workerProps); - metrics = new MockConnectMetrics(time); - recordsReturnedTp1 = 0; - recordsReturnedTp3 = 0; - } - - private void createTask(TargetState initialState) { - createTask(initialState, keyConverter, valueConverter, headerConverter); - } - - private void createTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter) { - createTask(initialState, keyConverter, valueConverter, headerConverter, RetryWithToleranceOperatorTest.noopOperator(), Collections::emptyList); - } - - private void createTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter, - RetryWithToleranceOperator> retryWithToleranceOperator, - Supplier>>> errorReportersSupplier) { - workerTask = new WorkerSinkTask( - taskId, sinkTask, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, metrics, - keyConverter, valueConverter, errorHandlingMetrics, headerConverter, - transformationChain, consumer, pluginLoader, time, - retryWithToleranceOperator, null, statusBackingStore, errorReportersSupplier); - } - - @After - public void tearDown() { - if (metrics != null) metrics.stop(); - } - - @Test - public void testStartPaused() { - createTask(TargetState.PAUSED); - - expectPollInitialAssignment(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - workerTask.iteration(); - verifyPollInitialAssignment(); - - time.sleep(10000L); - verify(consumer).pause(INITIAL_ASSIGNMENT); - - assertSinkMetricValue("partition-count", 2); - assertTaskMetricValue("status", "paused"); - assertTaskMetricValue("running-ratio", 0.0); - assertTaskMetricValue("pause-ratio", 1.0); - assertTaskMetricValue("offset-commit-max-time-ms", Double.NaN); - } - - @Test - public void testPause() { - createTask(initialState); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - expectTaskGetTopic(); - expectPollInitialAssignment() - .thenAnswer(expectConsumerPoll(1)) - // Pause - .thenThrow(new WakeupException()) - // Offset commit as requested when pausing; No records returned by consumer.poll() - .thenAnswer(expectConsumerPoll(0)) - // And unpause - .thenThrow(new WakeupException()) - .thenAnswer(expectConsumerPoll(1)); - - expectConversionAndTransformation(null, new RecordHeaders()); - - workerTask.iteration(); // initial assignment - verifyPollInitialAssignment(); - - workerTask.iteration(); // fetch some data - // put should've been called twice now (initial assignment & poll) - verify(sinkTask, times(2)).put(anyList()); - - workerTask.transitionTo(TargetState.PAUSED); - time.sleep(10_000L); - - assertSinkMetricValue("partition-count", 2); - assertSinkMetricValue("sink-record-read-total", 1.0); - assertSinkMetricValue("sink-record-send-total", 1.0); - assertSinkMetricValue("sink-record-active-count", 1.0); - assertSinkMetricValue("sink-record-active-count-max", 1.0); - assertSinkMetricValue("sink-record-active-count-avg", 0.333333); - assertSinkMetricValue("offset-commit-seq-no", 0.0); - assertSinkMetricValue("offset-commit-completion-rate", 0.0); - assertSinkMetricValue("offset-commit-completion-total", 0.0); - assertSinkMetricValue("offset-commit-skip-rate", 0.0); - assertSinkMetricValue("offset-commit-skip-total", 0.0); - assertTaskMetricValue("status", "running"); - assertTaskMetricValue("running-ratio", 1.0); - assertTaskMetricValue("pause-ratio", 0.0); - assertTaskMetricValue("batch-size-max", 1.0); - assertTaskMetricValue("batch-size-avg", 0.5); - assertTaskMetricValue("offset-commit-max-time-ms", Double.NaN); - assertTaskMetricValue("offset-commit-failure-percentage", 0.0); - assertTaskMetricValue("offset-commit-success-percentage", 0.0); - - workerTask.iteration(); // wakeup - // Pause - verify(statusListener).onPause(taskId); - verify(consumer).pause(INITIAL_ASSIGNMENT); - verify(consumer).wakeup(); - - // Offset commit as requested when pausing; No records returned by consumer.poll() - when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap()); - - workerTask.iteration(); // now paused - time.sleep(30000L); - - assertSinkMetricValue("offset-commit-seq-no", 1.0); - assertSinkMetricValue("offset-commit-completion-rate", 0.0333); - assertSinkMetricValue("offset-commit-completion-total", 1.0); - assertSinkMetricValue("offset-commit-skip-rate", 0.0); - assertSinkMetricValue("offset-commit-skip-total", 0.0); - assertTaskMetricValue("status", "paused"); - assertTaskMetricValue("running-ratio", 0.25); - assertTaskMetricValue("pause-ratio", 0.75); - verify(sinkTask, times(3)).put(anyList()); - - workerTask.transitionTo(TargetState.STARTED); - workerTask.iteration(); // wakeup - workerTask.iteration(); // now unpaused - - // And unpause - verify(statusListener).onResume(taskId); - verify(consumer, times(2)).wakeup(); - INITIAL_ASSIGNMENT.forEach(tp -> { - verify(consumer).resume(singleton(tp)); - }); - verify(sinkTask, times(4)).put(anyList()); - } - - @Test - public void testShutdown() throws Exception { - createTask(initialState); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - expectTaskGetTopic(); - expectPollInitialAssignment() - .thenAnswer(expectConsumerPoll(1)); - - expectConversionAndTransformation(null, new RecordHeaders()); - - workerTask.iteration(); - verifyPollInitialAssignment(); - sinkTaskContext.getValue().requestCommit(); // Force an offset commit - - // second iteration - when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap()); - - workerTask.iteration(); - verify(sinkTask, times(2)).put(anyList()); - - doAnswer((Answer>) invocation -> { - rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); - return null; - }).when(consumer).close(); - - workerTask.stop(); - verify(consumer).wakeup(); - - workerTask.close(); - verify(sinkTask).stop(); - verify(consumer).close(); - verify(headerConverter).close(); - } - - @Test - public void testPollRedelivery() { - createTask(initialState); - expectTaskGetTopic(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - expectPollInitialAssignment() - // If a retriable exception is thrown, we should redeliver the same batch, pausing the consumer in the meantime - .thenAnswer(expectConsumerPoll(1)) - // Retry delivery should succeed - .thenAnswer(expectConsumerPoll(0)) - .thenAnswer(expectConsumerPoll(0)); - expectConversionAndTransformation(null, new RecordHeaders()); - - doAnswer(invocation -> null) - .doThrow(new RetriableException("retry")) - .doAnswer(invocation -> null) - .when(sinkTask).put(anyList()); - - workerTask.iteration(); - time.sleep(10000L); - - verifyPollInitialAssignment(); - verify(sinkTask).put(anyList()); - - assertSinkMetricValue("partition-count", 2); - assertSinkMetricValue("sink-record-read-total", 0.0); - assertSinkMetricValue("sink-record-send-total", 0.0); - assertSinkMetricValue("sink-record-active-count", 0.0); - assertSinkMetricValue("sink-record-active-count-max", 0.0); - assertSinkMetricValue("sink-record-active-count-avg", 0.0); - assertSinkMetricValue("offset-commit-seq-no", 0.0); - assertSinkMetricValue("offset-commit-completion-rate", 0.0); - assertSinkMetricValue("offset-commit-completion-total", 0.0); - assertSinkMetricValue("offset-commit-skip-rate", 0.0); - assertSinkMetricValue("offset-commit-skip-total", 0.0); - assertTaskMetricValue("status", "running"); - assertTaskMetricValue("running-ratio", 1.0); - assertTaskMetricValue("pause-ratio", 0.0); - assertTaskMetricValue("batch-size-max", 0.0); - assertTaskMetricValue("batch-size-avg", 0.0); - assertTaskMetricValue("offset-commit-max-time-ms", Double.NaN); - assertTaskMetricValue("offset-commit-failure-percentage", 0.0); - assertTaskMetricValue("offset-commit-success-percentage", 0.0); - - // Pause - workerTask.iteration(); - - verify(consumer, times(3)).assignment(); - verify(consumer).pause(INITIAL_ASSIGNMENT); - - // Retry delivery should succeed - workerTask.iteration(); - time.sleep(30000L); - - verify(sinkTask, times(3)).put(anyList()); - INITIAL_ASSIGNMENT.forEach(tp -> { - verify(consumer).resume(Collections.singleton(tp)); - }); - - assertSinkMetricValue("sink-record-read-total", 1.0); - assertSinkMetricValue("sink-record-send-total", 1.0); - assertSinkMetricValue("sink-record-active-count", 1.0); - assertSinkMetricValue("sink-record-active-count-max", 1.0); - assertSinkMetricValue("sink-record-active-count-avg", 0.5); - assertTaskMetricValue("status", "running"); - assertTaskMetricValue("running-ratio", 1.0); - assertTaskMetricValue("batch-size-max", 1.0); - assertTaskMetricValue("batch-size-avg", 0.5); - - // Expect commit - final Map workerCurrentOffsets = new HashMap<>(); - // Commit advance by one - workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - // Nothing polled for this partition - workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - when(sinkTask.preCommit(workerCurrentOffsets)).thenReturn(workerCurrentOffsets); - - sinkTaskContext.getValue().requestCommit(); - time.sleep(10000L); - workerTask.iteration(); - - final ArgumentCaptor callback = ArgumentCaptor.forClass(OffsetCommitCallback.class); - verify(consumer).commitAsync(eq(workerCurrentOffsets), callback.capture()); - callback.getValue().onComplete(workerCurrentOffsets, null); - - verify(sinkTask, times(4)).put(anyList()); - assertSinkMetricValue("offset-commit-completion-total", 1.0); - } - - @Test - public void testErrorInRebalancePartitionLoss() { - RuntimeException exception = new RuntimeException("Revocation error"); - createTask(initialState); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - expectPollInitialAssignment() - .thenAnswer((Answer>) invocation -> { - rebalanceListener.getValue().onPartitionsLost(INITIAL_ASSIGNMENT); - return ConsumerRecords.empty(); - }); - - doThrow(exception).when(sinkTask).close(INITIAL_ASSIGNMENT); - - workerTask.iteration(); - verifyPollInitialAssignment(); - - try { - workerTask.iteration(); - fail("Poll should have raised the rebalance exception"); - } catch (RuntimeException e) { - assertEquals(exception, e); - } - } - - @Test - public void testErrorInRebalancePartitionRevocation() { - RuntimeException exception = new RuntimeException("Revocation error"); - createTask(initialState); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - expectPollInitialAssignment() - .thenAnswer((Answer>) invocation -> { - rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); - return ConsumerRecords.empty(); - }); - - expectRebalanceRevocationError(exception); - - workerTask.iteration(); - verifyPollInitialAssignment(); - try { - workerTask.iteration(); - fail("Poll should have raised the rebalance exception"); - } catch (RuntimeException e) { - assertEquals(exception, e); - } - } - - @Test - public void testErrorInRebalancePartitionAssignment() { - RuntimeException exception = new RuntimeException("Assignment error"); - createTask(initialState); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - expectPollInitialAssignment() - .thenAnswer((Answer>) invocation -> { - rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); - rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); - return ConsumerRecords.empty(); - }); - - workerTask.iteration(); - verifyPollInitialAssignment(); - - expectRebalanceAssignmentError(exception); - try { - workerTask.iteration(); - fail("Poll should have raised the rebalance exception"); - } catch (RuntimeException e) { - assertEquals(exception, e); - } finally { - verify(sinkTask).close(INITIAL_ASSIGNMENT); - } - } - - @Test - public void testPartialRevocationAndAssignment() { - createTask(initialState); - - when(consumer.assignment()) - .thenReturn(INITIAL_ASSIGNMENT) - .thenReturn(INITIAL_ASSIGNMENT) - .thenReturn(Collections.singleton(TOPIC_PARTITION2)) - .thenReturn(Collections.singleton(TOPIC_PARTITION2)) - .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))) - .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))) - .thenReturn(INITIAL_ASSIGNMENT) - .thenReturn(INITIAL_ASSIGNMENT) - .thenReturn(INITIAL_ASSIGNMENT); - - INITIAL_ASSIGNMENT.forEach(tp -> when(consumer.position(tp)).thenReturn(FIRST_OFFSET)); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - when(consumer.poll(any(Duration.class))) - .thenAnswer((Answer>) invocation -> { - rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); - return ConsumerRecords.empty(); - }) - .thenAnswer((Answer>) invocation -> { - rebalanceListener.getValue().onPartitionsRevoked(singleton(TOPIC_PARTITION)); - rebalanceListener.getValue().onPartitionsAssigned(Collections.emptySet()); - return ConsumerRecords.empty(); - }) - .thenAnswer((Answer>) invocation -> { - rebalanceListener.getValue().onPartitionsRevoked(Collections.emptySet()); - rebalanceListener.getValue().onPartitionsAssigned(singleton(TOPIC_PARTITION3)); - return ConsumerRecords.empty(); - }) - .thenAnswer((Answer>) invocation -> { - rebalanceListener.getValue().onPartitionsLost(singleton(TOPIC_PARTITION3)); - rebalanceListener.getValue().onPartitionsAssigned(singleton(TOPIC_PARTITION)); - return ConsumerRecords.empty(); - }); - - final Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); - when(sinkTask.preCommit(offsets)).thenReturn(offsets); - - when(consumer.position(TOPIC_PARTITION3)).thenReturn(FIRST_OFFSET); - - // First iteration--first call to poll, first consumer assignment - workerTask.iteration(); - verifyPollInitialAssignment(); - - // Second iteration--second call to poll, partial consumer revocation - workerTask.iteration(); - verify(sinkTask).close(singleton(TOPIC_PARTITION)); - verify(sinkTask, times(2)).put(Collections.emptyList()); - - // Third iteration--third call to poll, partial consumer assignment - workerTask.iteration(); - verify(sinkTask).open(singleton(TOPIC_PARTITION3)); - verify(sinkTask, times(3)).put(Collections.emptyList()); - - // Fourth iteration--fourth call to poll, one partition lost; can't commit offsets for it, one new partition assigned - workerTask.iteration(); - verify(sinkTask).close(singleton(TOPIC_PARTITION3)); - verify(sinkTask).open(singleton(TOPIC_PARTITION)); - verify(sinkTask, times(4)).put(Collections.emptyList()); - } - - @SuppressWarnings("unchecked") - @Test - public void testTaskCancelPreventsFinalOffsetCommit() { - createTask(initialState); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - expectTaskGetTopic(); - expectPollInitialAssignment() - // Put one message through the task to get some offsets to commit - .thenAnswer(expectConsumerPoll(1)) - // the second put will return after the task is stopped and cancelled (asynchronously) - .thenAnswer(expectConsumerPoll(1)); - - expectConversionAndTransformation(null, new RecordHeaders()); - - doAnswer(invocation -> null) - .doAnswer(invocation -> null) - .doAnswer(invocation -> { - workerTask.stop(); - workerTask.cancel(); - return null; - }) - .when(sinkTask).put(anyList()); - - // task performs normal steps in advance of committing offsets - final Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 2)); - offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - when(sinkTask.preCommit(offsets)).thenReturn(offsets); - - workerTask.execute(); - - // stop wakes up the consumer - verify(consumer).wakeup(); - - verify(sinkTask).close(any()); - } - - @Test - public void testDeliveryWithMutatingTransform() { - createTask(initialState); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - expectTaskGetTopic(); - expectPollInitialAssignment() - .thenAnswer(expectConsumerPoll(1)) - .thenAnswer(expectConsumerPoll(0)); - - expectConversionAndTransformation("newtopic_", new RecordHeaders()); - - workerTask.iteration(); // initial assignment - - workerTask.iteration(); // first record delivered - - final Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - when(sinkTask.preCommit(offsets)).thenReturn(offsets); - - sinkTaskContext.getValue().requestCommit(); - assertTrue(sinkTaskContext.getValue().isCommitRequested()); - - assertNotEquals(offsets, workerTask.lastCommittedOffsets()); - workerTask.iteration(); // triggers the commit - - ArgumentCaptor callback = ArgumentCaptor.forClass(OffsetCommitCallback.class); - verify(consumer).commitAsync(eq(offsets), callback.capture()); - - callback.getValue().onComplete(offsets, null); - - assertFalse(sinkTaskContext.getValue().isCommitRequested()); // should have been cleared - assertEquals(offsets, workerTask.lastCommittedOffsets()); - assertEquals(0, workerTask.commitFailures()); - assertEquals(1.0, metrics.currentMetricValueAsDouble(workerTask.taskMetricsGroup().metricGroup(), "batch-size-max"), 0.0001); - } - - @Test - public void testMissingTimestampPropagation() { - createTask(initialState); - expectTaskGetTopic(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - expectPollInitialAssignment() - .thenAnswer(expectConsumerPoll(1, RecordBatch.NO_TIMESTAMP, TimestampType.CREATE_TIME, new RecordHeaders())); - - expectConversionAndTransformation(null, new RecordHeaders()); - - workerTask.iteration(); // iter 1 -- initial assignment - workerTask.iteration(); // iter 2 -- deliver 1 record - - @SuppressWarnings("unchecked") - ArgumentCaptor> records = ArgumentCaptor.forClass(Collection.class); - verify(sinkTask, times(2)).put(records.capture()); - - SinkRecord record = records.getValue().iterator().next(); - - // we expect null for missing timestamp, the sentinel value of Record.NO_TIMESTAMP is Kafka's API - assertNull(record.timestamp()); - assertEquals(TimestampType.CREATE_TIME, record.timestampType()); - } - - @Test - public void testTimestampPropagation() { - final Long timestamp = System.currentTimeMillis(); - final TimestampType timestampType = TimestampType.CREATE_TIME; - - createTask(initialState); - expectTaskGetTopic(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - expectPollInitialAssignment() - .thenAnswer(expectConsumerPoll(1, timestamp, timestampType, new RecordHeaders())); - - expectConversionAndTransformation(null, new RecordHeaders()); - - workerTask.iteration(); // iter 1 -- initial assignment - workerTask.iteration(); // iter 2 -- deliver 1 record - - @SuppressWarnings("unchecked") - ArgumentCaptor> records = ArgumentCaptor.forClass(Collection.class); - verify(sinkTask, times(2)).put(records.capture()); - - SinkRecord record = records.getValue().iterator().next(); - - assertEquals(timestamp, record.timestamp()); - assertEquals(timestampType, record.timestampType()); - } - - @Test - public void testTopicsRegex() { - Map props = new HashMap<>(TASK_PROPS); - props.remove("topics"); - props.put("topics.regex", "te.*"); - TaskConfig taskConfig = new TaskConfig(props); - - createTask(TargetState.PAUSED); - - workerTask.initialize(taskConfig); - workerTask.initializeAndStart(); - - ArgumentCaptor topicsRegex = ArgumentCaptor.forClass(Pattern.class); - - verify(consumer).subscribe(topicsRegex.capture(), rebalanceListener.capture()); - assertEquals("te.*", topicsRegex.getValue().pattern()); - verify(sinkTask).initialize(sinkTaskContext.capture()); - verify(sinkTask).start(props); - - expectPollInitialAssignment(); - - workerTask.iteration(); - time.sleep(10000L); - - verify(consumer).pause(INITIAL_ASSIGNMENT); - } - - @Test - public void testMetricsGroup() { - SinkTaskMetricsGroup group = new SinkTaskMetricsGroup(taskId, metrics); - SinkTaskMetricsGroup group1 = new SinkTaskMetricsGroup(taskId1, metrics); - for (int i = 0; i != 10; ++i) { - group.recordRead(1); - group.recordSend(2); - group.recordPut(3); - group.recordPartitionCount(4); - group.recordOffsetSequenceNumber(5); - } - Map committedOffsets = new HashMap<>(); - committedOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - group.recordCommittedOffsets(committedOffsets); - Map consumedOffsets = new HashMap<>(); - consumedOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 10)); - group.recordConsumedOffsets(consumedOffsets); - - for (int i = 0; i != 20; ++i) { - group1.recordRead(1); - group1.recordSend(2); - group1.recordPut(30); - group1.recordPartitionCount(40); - group1.recordOffsetSequenceNumber(50); - } - committedOffsets = new HashMap<>(); - committedOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET + 2)); - committedOffsets.put(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET + 3)); - group1.recordCommittedOffsets(committedOffsets); - consumedOffsets = new HashMap<>(); - consumedOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET + 20)); - consumedOffsets.put(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET + 30)); - group1.recordConsumedOffsets(consumedOffsets); - - assertEquals(0.333, metrics.currentMetricValueAsDouble(group.metricGroup(), "sink-record-read-rate"), 0.001d); - assertEquals(0.667, metrics.currentMetricValueAsDouble(group.metricGroup(), "sink-record-send-rate"), 0.001d); - assertEquals(9, metrics.currentMetricValueAsDouble(group.metricGroup(), "sink-record-active-count"), 0.001d); - assertEquals(4, metrics.currentMetricValueAsDouble(group.metricGroup(), "partition-count"), 0.001d); - assertEquals(5, metrics.currentMetricValueAsDouble(group.metricGroup(), "offset-commit-seq-no"), 0.001d); - assertEquals(3, metrics.currentMetricValueAsDouble(group.metricGroup(), "put-batch-max-time-ms"), 0.001d); - - // Close the group - group.close(); - - for (MetricName metricName : group.metricGroup().metrics().metrics().keySet()) { - // Metrics for this group should no longer exist - assertFalse(group.metricGroup().groupId().includes(metricName)); - } - // Sensors for this group should no longer exist - assertNull(group.metricGroup().metrics().getSensor("source-record-poll")); - assertNull(group.metricGroup().metrics().getSensor("source-record-write")); - assertNull(group.metricGroup().metrics().getSensor("poll-batch-time")); - - assertEquals(0.667, metrics.currentMetricValueAsDouble(group1.metricGroup(), "sink-record-read-rate"), 0.001d); - assertEquals(1.333, metrics.currentMetricValueAsDouble(group1.metricGroup(), "sink-record-send-rate"), 0.001d); - assertEquals(45, metrics.currentMetricValueAsDouble(group1.metricGroup(), "sink-record-active-count"), 0.001d); - assertEquals(40, metrics.currentMetricValueAsDouble(group1.metricGroup(), "partition-count"), 0.001d); - assertEquals(50, metrics.currentMetricValueAsDouble(group1.metricGroup(), "offset-commit-seq-no"), 0.001d); - assertEquals(30, metrics.currentMetricValueAsDouble(group1.metricGroup(), "put-batch-max-time-ms"), 0.001d); - } - - @Test - public void testHeaders() { - createTask(initialState); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - Headers headers = new RecordHeaders(); - headers.add("header_key", "header_value".getBytes()); - - expectPollInitialAssignment() - .thenAnswer(expectConsumerPoll(1, headers)); - - expectConversionAndTransformation(null, headers); - - workerTask.iteration(); // iter 1 -- initial assignment - workerTask.iteration(); // iter 2 -- deliver 1 record - - @SuppressWarnings("unchecked") - ArgumentCaptor> recordCapture = ArgumentCaptor.forClass(Collection.class); - verify(sinkTask, times(2)).put(recordCapture.capture()); - - assertEquals(1, recordCapture.getValue().size()); - SinkRecord record = recordCapture.getValue().iterator().next(); - - assertEquals("header_value", record.headers().lastWithName("header_key").value()); - } - - @Test - public void testHeadersWithCustomConverter() { - StringConverter stringConverter = new StringConverter(); - SampleConverterWithHeaders testConverter = new SampleConverterWithHeaders(); - - createTask(initialState, stringConverter, testConverter, stringConverter); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - String keyA = "a"; - String valueA = "Árvíztűrő tükörfúrógép"; - Headers headersA = new RecordHeaders(); - String encodingA = "latin2"; - headersA.add("encoding", encodingA.getBytes()); - - String keyB = "b"; - String valueB = "Тестовое сообщение"; - Headers headersB = new RecordHeaders(); - String encodingB = "koi8_r"; - headersB.add("encoding", encodingB.getBytes()); - - expectPollInitialAssignment() - .thenAnswer((Answer>) invocation -> { - List> records = Arrays.asList( - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, - 0, 0, keyA.getBytes(), valueA.getBytes(encodingA), headersA, Optional.empty()), - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 2, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, - 0, 0, keyB.getBytes(), valueB.getBytes(encodingB), headersB, Optional.empty()) - ); - return new ConsumerRecords<>(Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records)); - }); - - expectTransformation(null); - - workerTask.iteration(); // iter 1 -- initial assignment - workerTask.iteration(); // iter 2 -- deliver records - - @SuppressWarnings("unchecked") - ArgumentCaptor> records = ArgumentCaptor.forClass(Collection.class); - verify(sinkTask, times(2)).put(records.capture()); - - Iterator iterator = records.getValue().iterator(); - - SinkRecord recordA = iterator.next(); - assertEquals(keyA, recordA.key()); - assertEquals(valueA, recordA.value()); - - SinkRecord recordB = iterator.next(); - assertEquals(keyB, recordB.key()); - assertEquals(valueB, recordB.value()); - } - - @Test - public void testOriginalTopicWithTopicMutatingTransformations() { - createTask(initialState); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - verifyInitializeTask(); - - expectPollInitialAssignment() - .thenAnswer(expectConsumerPoll(1)); - - expectConversionAndTransformation("newtopic_", new RecordHeaders()); - - workerTask.iteration(); // initial assignment - workerTask.iteration(); // first record delivered - - @SuppressWarnings("unchecked") - ArgumentCaptor> recordCapture = ArgumentCaptor.forClass(Collection.class); - verify(sinkTask, times(2)).put(recordCapture.capture()); - - assertEquals(1, recordCapture.getValue().size()); - SinkRecord record = recordCapture.getValue().iterator().next(); - assertEquals(TOPIC, record.originalTopic()); - assertEquals("newtopic_" + TOPIC, record.topic()); - } - - @Test - public void testPartitionCountInCaseOfPartitionRevocation() { - MockConsumer mockConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); - // Setting up Worker Sink Task to check metrics - workerTask = new WorkerSinkTask( - taskId, sinkTask, statusListener, TargetState.PAUSED, workerConfig, ClusterConfigState.EMPTY, metrics, - keyConverter, valueConverter, errorHandlingMetrics, headerConverter, - transformationChain, mockConsumer, pluginLoader, time, - RetryWithToleranceOperatorTest.noopOperator(), null, statusBackingStore, Collections::emptyList); - mockConsumer.updateBeginningOffsets( - new HashMap() {{ - put(TOPIC_PARTITION, 0L); - put(TOPIC_PARTITION2, 0L); - }} - ); - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - // Initial Re-balance to assign INITIAL_ASSIGNMENT which is "TOPIC_PARTITION" and "TOPIC_PARTITION2" - mockConsumer.rebalance(INITIAL_ASSIGNMENT); - assertSinkMetricValue("partition-count", 2); - // Revoked "TOPIC_PARTITION" and second re-balance with "TOPIC_PARTITION2" - mockConsumer.rebalance(Collections.singleton(TOPIC_PARTITION2)); - assertSinkMetricValue("partition-count", 1); - // Closing the Worker Sink Task which will update the partition count as 0. - workerTask.close(); - assertSinkMetricValue("partition-count", 0); - } - - private void expectRebalanceRevocationError(RuntimeException e) { - when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap()); - doThrow(e).when(sinkTask).close(INITIAL_ASSIGNMENT); - } - - private void expectRebalanceAssignmentError(RuntimeException e) { - when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap()); - when(consumer.position(TOPIC_PARTITION)).thenReturn(FIRST_OFFSET); - when(consumer.position(TOPIC_PARTITION2)).thenReturn(FIRST_OFFSET); - - doThrow(e).when(sinkTask).open(INITIAL_ASSIGNMENT); - } - - private void verifyInitializeTask() { - verify(consumer).subscribe(eq(asList(TOPIC)), rebalanceListener.capture()); - verify(sinkTask).initialize(sinkTaskContext.capture()); - verify(sinkTask).start(TASK_PROPS); - } - - private OngoingStubbing> expectPollInitialAssignment() { - when(consumer.assignment()).thenReturn(INITIAL_ASSIGNMENT); - INITIAL_ASSIGNMENT.forEach(tp -> when(consumer.position(tp)).thenReturn(FIRST_OFFSET)); - - return when(consumer.poll(any(Duration.class))).thenAnswer( - invocation -> { - rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); - return ConsumerRecords.empty(); - } - ); - } - - private void verifyPollInitialAssignment() { - verify(sinkTask).open(INITIAL_ASSIGNMENT); - verify(consumer, atLeastOnce()).assignment(); - verify(sinkTask).put(Collections.emptyList()); - } - - private Answer> expectConsumerPoll(final int numMessages) { - return expectConsumerPoll(numMessages, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, new RecordHeaders()); - } - - private Answer> expectConsumerPoll(final int numMessages, Headers headers) { - return expectConsumerPoll(numMessages, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, headers); - } - - private Answer> expectConsumerPoll(final int numMessages, final long timestamp, final TimestampType timestampType, Headers headers) { - return invocation -> { - List> records = new ArrayList<>(); - for (int i = 0; i < numMessages; i++) - records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + i, timestamp, timestampType, - 0, 0, RAW_KEY, RAW_VALUE, headers, Optional.empty())); - recordsReturnedTp1 += numMessages; - return new ConsumerRecords<>( - numMessages > 0 ? - Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records) - : Collections.emptyMap() - ); - }; - } - - private void expectConversionAndTransformation(final String topicPrefix, final Headers headers) { - when(keyConverter.toConnectData(TOPIC, headers, RAW_KEY)).thenReturn(new SchemaAndValue(KEY_SCHEMA, KEY)); - when(valueConverter.toConnectData(TOPIC, headers, RAW_VALUE)).thenReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)); - - for (Header header : headers) { - when(headerConverter.toConnectHeader(TOPIC, header.key(), header.value())).thenReturn(new SchemaAndValue(VALUE_SCHEMA, new String(header.value()))); - } - - expectTransformation(topicPrefix); - } - - @SuppressWarnings("unchecked") - private void expectTransformation(final String topicPrefix) { - when(transformationChain.apply(any(ProcessingContext.class), any(SinkRecord.class))).thenAnswer((Answer) - invocation -> { - SinkRecord origRecord = invocation.getArgument(1); - return topicPrefix != null && !topicPrefix.isEmpty() - ? origRecord.newRecord( - topicPrefix + origRecord.topic(), - origRecord.kafkaPartition(), - origRecord.keySchema(), - origRecord.key(), - origRecord.valueSchema(), - origRecord.value(), - origRecord.timestamp(), - origRecord.headers() - ) : origRecord; - }); - } - - private void expectTaskGetTopic() { - when(statusBackingStore.getTopic(anyString(), anyString())).thenAnswer((Answer) invocation -> { - String connector = invocation.getArgument(0, String.class); - String topic = invocation.getArgument(1, String.class); - return new TopicStatus(topic, new ConnectorTaskId(connector, 0), Time.SYSTEM.milliseconds()); - }); - } - - private void assertSinkMetricValue(String name, double expected) { - MetricGroup sinkTaskGroup = workerTask.sinkTaskMetricsGroup().metricGroup(); - double measured = metrics.currentMetricValueAsDouble(sinkTaskGroup, name); - assertEquals(expected, measured, 0.001d); - } - - private void assertTaskMetricValue(String name, double expected) { - MetricGroup taskGroup = workerTask.taskMetricsGroup().metricGroup(); - double measured = metrics.currentMetricValueAsDouble(taskGroup, name); - assertEquals(expected, measured, 0.001d); - } - - private void assertTaskMetricValue(String name, String expected) { - MetricGroup taskGroup = workerTask.taskMetricsGroup().metricGroup(); - String measured = metrics.currentMetricValueAsString(taskGroup, name); - assertEquals(expected, measured); - } -} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java index e103c30157..21b2b10c16 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java @@ -16,12 +16,54 @@ */ package org.apache.kafka.connect.runtime; +import static java.util.Arrays.asList; +import static java.util.Collections.singleton; +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.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.anyMap; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.regex.Pattern; +import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; 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.consumer.MockConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.header.Header; @@ -36,8 +78,10 @@ import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.RetriableException; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; +import org.apache.kafka.connect.runtime.WorkerSinkTask.SinkTaskMetricsGroup; import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics; import org.apache.kafka.connect.runtime.errors.ErrorReporter; +import org.apache.kafka.connect.runtime.errors.ProcessingContext; import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest; import org.apache.kafka.connect.runtime.isolation.PluginClassLoader; @@ -49,52 +93,23 @@ import org.apache.kafka.connect.storage.Converter; import org.apache.kafka.connect.storage.HeaderConverter; import org.apache.kafka.connect.storage.StatusBackingStore; +import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.util.ConnectorTaskId; -import org.easymock.Capture; -import org.easymock.EasyMock; -import org.easymock.IExpectationSetters; import org.junit.After; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; -import org.powermock.api.easymock.PowerMock; -import org.powermock.api.easymock.annotation.Mock; -import org.powermock.core.classloader.annotations.PowerMockIgnore; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; -import org.powermock.reflect.Whitebox; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; -import java.util.function.Supplier; -import java.util.stream.Collectors; - -import static java.util.Arrays.asList; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -@RunWith(PowerMockRunner.class) -@PrepareForTest(WorkerSinkTask.class) -@PowerMockIgnore("javax.management.*") +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoJUnitRunner; +import org.mockito.junit.MockitoRule; +import org.mockito.quality.Strictness; +import org.mockito.stubbing.Answer; +import org.mockito.stubbing.OngoingStubbing; + +@RunWith(MockitoJUnitRunner.StrictStubs.class) public class WorkerSinkTaskTest { // These are fixed to keep this code simpler. In this example we assume byte[] raw values // with mix of integer/string in Connect @@ -115,22 +130,25 @@ public class WorkerSinkTaskTest { private static final TopicPartition TOPIC_PARTITION3 = new TopicPartition(TOPIC, PARTITION3); private static final Set INITIAL_ASSIGNMENT = - new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2)); + new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2)); private static final Map TASK_PROPS = new HashMap<>(); + static { TASK_PROPS.put(SinkConnector.TOPICS_CONFIG, TOPIC); - TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName()); + TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, SinkTask.class.getName()); } + private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private ConnectorTaskId taskId1 = new ConnectorTaskId("job", 1); private TargetState initialState = TargetState.STARTED; private MockTime time; private WorkerSinkTask workerTask; @Mock private SinkTask sinkTask; - private Capture sinkTaskContext = EasyMock.newCapture(); + private ArgumentCaptor sinkTaskContext = ArgumentCaptor.forClass(WorkerSinkTaskContext.class); private WorkerConfig workerConfig; private MockConnectMetrics metrics; @Mock @@ -151,7 +169,9 @@ public class WorkerSinkTaskTest { private KafkaConsumer consumer; @Mock private ErrorHandlingMetrics errorHandlingMetrics; - private Capture rebalanceListener = EasyMock.newCapture(); + private ArgumentCaptor rebalanceListener = ArgumentCaptor.forClass(ConsumerRebalanceListener.class); + @Rule + public final MockitoRule rule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); private long recordsReturnedTp1; private long recordsReturnedTp3; @@ -164,7 +184,6 @@ public void setUp() { workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); workerConfig = new StandaloneConfig(workerProps); - pluginLoader = PowerMock.createMock(PluginClassLoader.class); metrics = new MockConnectMetrics(time); recordsReturnedTp1 = 0; recordsReturnedTp3 = 0; @@ -179,7 +198,8 @@ private void createTask(TargetState initialState, Converter keyConverter, Conver } private void createTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter, - RetryWithToleranceOperator> retryWithToleranceOperator, Supplier>>> errorReportersSupplier) { + RetryWithToleranceOperator> retryWithToleranceOperator, + Supplier>>> errorReportersSupplier) { workerTask = new WorkerSinkTask( taskId, sinkTask, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, metrics, keyConverter, valueConverter, errorHandlingMetrics, headerConverter, @@ -193,232 +213,590 @@ public void tearDown() { } @Test - public void testPollRedeliveryWithConsumerRebalance() throws Exception { - createTask(initialState); + public void testStartPaused() { + createTask(TargetState.PAUSED); - expectInitializeTask(); - expectTaskGetTopic(true); expectPollInitialAssignment(); - // If a retriable exception is thrown, we should redeliver the same batch, pausing the consumer in the meantime - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall().andThrow(new RetriableException("retry")); + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + workerTask.iteration(); + verifyPollInitialAssignment(); + + time.sleep(10000L); + verify(consumer).pause(INITIAL_ASSIGNMENT); + + assertSinkMetricValue("partition-count", 2); + assertTaskMetricValue("status", "paused"); + assertTaskMetricValue("running-ratio", 0.0); + assertTaskMetricValue("pause-ratio", 1.0); + assertTaskMetricValue("offset-commit-max-time-ms", Double.NaN); + } + + @Test + public void testPause() { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1)) + // Pause + .thenThrow(new WakeupException()) + // Offset commit as requested when pausing; No records returned by consumer.poll() + .thenAnswer(expectConsumerPoll(0)) + // And unpause + .thenThrow(new WakeupException()) + .thenAnswer(expectConsumerPoll(1)); + + expectConversionAndTransformation(null, new RecordHeaders()); + + workerTask.iteration(); // initial assignment + verifyPollInitialAssignment(); + + workerTask.iteration(); // fetch some data + // put should've been called twice now (initial assignment & poll) + verify(sinkTask, times(2)).put(anyList()); + + workerTask.transitionTo(TargetState.PAUSED); + time.sleep(10_000L); + + assertSinkMetricValue("partition-count", 2); + assertSinkMetricValue("sink-record-read-total", 1.0); + assertSinkMetricValue("sink-record-send-total", 1.0); + assertSinkMetricValue("sink-record-active-count", 1.0); + assertSinkMetricValue("sink-record-active-count-max", 1.0); + assertSinkMetricValue("sink-record-active-count-avg", 0.333333); + assertSinkMetricValue("offset-commit-seq-no", 0.0); + assertSinkMetricValue("offset-commit-completion-rate", 0.0); + assertSinkMetricValue("offset-commit-completion-total", 0.0); + assertSinkMetricValue("offset-commit-skip-rate", 0.0); + assertSinkMetricValue("offset-commit-skip-total", 0.0); + assertTaskMetricValue("status", "running"); + assertTaskMetricValue("running-ratio", 1.0); + assertTaskMetricValue("pause-ratio", 0.0); + assertTaskMetricValue("batch-size-max", 1.0); + assertTaskMetricValue("batch-size-avg", 0.5); + assertTaskMetricValue("offset-commit-max-time-ms", Double.NaN); + assertTaskMetricValue("offset-commit-failure-percentage", 0.0); + assertTaskMetricValue("offset-commit-success-percentage", 0.0); + + workerTask.iteration(); // wakeup // Pause - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT); - consumer.pause(INITIAL_ASSIGNMENT); - PowerMock.expectLastCall(); - - // Empty consumer poll (all partitions are paused) with rebalance; one new partition is assigned - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { - rebalanceListener.getValue().onPartitionsRevoked(Collections.emptySet()); - rebalanceListener.getValue().onPartitionsAssigned(Collections.singleton(TOPIC_PARTITION3)); - return ConsumerRecords.empty(); - }); - Set newAssignment = new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3)); - EasyMock.expect(consumer.assignment()).andReturn(newAssignment).times(3); - EasyMock.expect(consumer.position(TOPIC_PARTITION3)).andReturn(FIRST_OFFSET); - sinkTask.open(Collections.singleton(TOPIC_PARTITION3)); - EasyMock.expectLastCall(); - // All partitions are re-paused in order to pause any newly-assigned partitions so that redelivery efforts can continue - consumer.pause(newAssignment); - EasyMock.expectLastCall(); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall().andThrow(new RetriableException("retry")); - - // Next delivery attempt fails again - expectConsumerPoll(0); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall().andThrow(new RetriableException("retry")); - - // Non-empty consumer poll; all initially-assigned partitions are revoked in rebalance, and new partitions are allowed to resume - ConsumerRecord newRecord = new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET, RAW_KEY, RAW_VALUE); - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { - rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); - rebalanceListener.getValue().onPartitionsAssigned(Collections.emptyList()); - return new ConsumerRecords<>(Collections.singletonMap(TOPIC_PARTITION3, Collections.singletonList(newRecord))); - }); - newAssignment = Collections.singleton(TOPIC_PARTITION3); - EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(newAssignment)).times(3); - final Map offsets = INITIAL_ASSIGNMENT.stream() - .collect(Collectors.toMap(Function.identity(), tp -> new OffsetAndMetadata(FIRST_OFFSET))); - sinkTask.preCommit(offsets); - EasyMock.expectLastCall().andReturn(offsets); - sinkTask.close(INITIAL_ASSIGNMENT); - EasyMock.expectLastCall(); - // All partitions are resumed, as all previously paused-for-redelivery partitions were revoked - newAssignment.forEach(tp -> { - consumer.resume(Collections.singleton(tp)); - EasyMock.expectLastCall(); - }); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); + verify(statusListener).onPause(taskId); + verify(consumer).pause(INITIAL_ASSIGNMENT); + verify(consumer).wakeup(); + + // Offset commit as requested when pausing; No records returned by consumer.poll() + when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap()); + + workerTask.iteration(); // now paused + time.sleep(30000L); - PowerMock.replayAll(); + assertSinkMetricValue("offset-commit-seq-no", 1.0); + assertSinkMetricValue("offset-commit-completion-rate", 0.0333); + assertSinkMetricValue("offset-commit-completion-total", 1.0); + assertSinkMetricValue("offset-commit-skip-rate", 0.0); + assertSinkMetricValue("offset-commit-skip-total", 0.0); + assertTaskMetricValue("status", "paused"); + assertTaskMetricValue("running-ratio", 0.25); + assertTaskMetricValue("pause-ratio", 0.75); + verify(sinkTask, times(3)).put(anyList()); + + workerTask.transitionTo(TargetState.STARTED); + workerTask.iteration(); // wakeup + workerTask.iteration(); // now unpaused + + // And unpause + verify(statusListener).onResume(taskId); + verify(consumer, times(2)).wakeup(); + INITIAL_ASSIGNMENT.forEach(tp -> verify(consumer).resume(singleton(tp))); + verify(sinkTask, times(4)).put(anyList()); + } + + @Test + public void testShutdown() throws Exception { + createTask(initialState); workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1)); + + expectConversionAndTransformation(null, new RecordHeaders()); + workerTask.iteration(); + verifyPollInitialAssignment(); + sinkTaskContext.getValue().requestCommit(); // Force an offset commit + + // second iteration + when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap()); + workerTask.iteration(); + verify(sinkTask, times(2)).put(anyList()); + + doAnswer((Answer>) invocation -> { + rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); + return null; + }).when(consumer).close(); + + workerTask.stop(); + verify(consumer).wakeup(); + + workerTask.close(); + verify(sinkTask).stop(); + verify(consumer).close(); + verify(headerConverter).close(); + } + + @Test + public void testPollRedelivery() { + createTask(initialState); + expectTaskGetTopic(); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectPollInitialAssignment() + // If a retriable exception is thrown, we should redeliver the same batch, pausing the consumer in the meantime + .thenAnswer(expectConsumerPoll(1)) + // Retry delivery should succeed + .thenAnswer(expectConsumerPoll(0)) + .thenAnswer(expectConsumerPoll(0)); + expectConversionAndTransformation(null, new RecordHeaders()); + + doNothing() + .doThrow(new RetriableException("retry")) + .doNothing() + .when(sinkTask).put(anyList()); + workerTask.iteration(); + time.sleep(10000L); + + verifyPollInitialAssignment(); + verify(sinkTask).put(anyList()); + + assertSinkMetricValue("partition-count", 2); + assertSinkMetricValue("sink-record-read-total", 0.0); + assertSinkMetricValue("sink-record-send-total", 0.0); + assertSinkMetricValue("sink-record-active-count", 0.0); + assertSinkMetricValue("sink-record-active-count-max", 0.0); + assertSinkMetricValue("sink-record-active-count-avg", 0.0); + assertSinkMetricValue("offset-commit-seq-no", 0.0); + assertSinkMetricValue("offset-commit-completion-rate", 0.0); + assertSinkMetricValue("offset-commit-completion-total", 0.0); + assertSinkMetricValue("offset-commit-skip-rate", 0.0); + assertSinkMetricValue("offset-commit-skip-total", 0.0); + assertTaskMetricValue("status", "running"); + assertTaskMetricValue("running-ratio", 1.0); + assertTaskMetricValue("pause-ratio", 0.0); + assertTaskMetricValue("batch-size-max", 0.0); + assertTaskMetricValue("batch-size-avg", 0.0); + assertTaskMetricValue("offset-commit-max-time-ms", Double.NaN); + assertTaskMetricValue("offset-commit-failure-percentage", 0.0); + assertTaskMetricValue("offset-commit-success-percentage", 0.0); + + // Pause workerTask.iteration(); + + verify(consumer, times(3)).assignment(); + verify(consumer).pause(INITIAL_ASSIGNMENT); + + // Retry delivery should succeed workerTask.iteration(); + time.sleep(30000L); - PowerMock.verifyAll(); + verify(sinkTask, times(3)).put(anyList()); + INITIAL_ASSIGNMENT.forEach(tp -> verify(consumer).resume(Collections.singleton(tp))); + + assertSinkMetricValue("sink-record-read-total", 1.0); + assertSinkMetricValue("sink-record-send-total", 1.0); + assertSinkMetricValue("sink-record-active-count", 1.0); + assertSinkMetricValue("sink-record-active-count-max", 1.0); + assertSinkMetricValue("sink-record-active-count-avg", 0.5); + assertTaskMetricValue("status", "running"); + assertTaskMetricValue("running-ratio", 1.0); + assertTaskMetricValue("batch-size-max", 1.0); + assertTaskMetricValue("batch-size-avg", 0.5); + + // Expect commit + final Map workerCurrentOffsets = new HashMap<>(); + // Commit advance by one + workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + // Nothing polled for this partition + workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + when(sinkTask.preCommit(workerCurrentOffsets)).thenReturn(workerCurrentOffsets); + + sinkTaskContext.getValue().requestCommit(); + time.sleep(10000L); + workerTask.iteration(); + + final ArgumentCaptor callback = ArgumentCaptor.forClass(OffsetCommitCallback.class); + verify(consumer).commitAsync(eq(workerCurrentOffsets), callback.capture()); + callback.getValue().onComplete(workerCurrentOffsets, null); + + verify(sinkTask, times(4)).put(anyList()); + assertSinkMetricValue("offset-commit-completion-total", 1.0); } @Test - public void testPreCommitFailureAfterPartialRevocationAndAssignment() throws Exception { + @SuppressWarnings("unchecked") + public void testPollRedeliveryWithConsumerRebalance() { createTask(initialState); + expectTaskGetTopic(); - // First poll; assignment is [TP1, TP2] - expectInitializeTask(); - expectTaskGetTopic(true); - expectPollInitialAssignment(); + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); - // Second poll; a single record is delivered from TP1 - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); + Set newAssignment = new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3)); - // Third poll; assignment changes to [TP2] - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { - rebalanceListener.getValue().onPartitionsRevoked(Collections.singleton(TOPIC_PARTITION)); - rebalanceListener.getValue().onPartitionsAssigned(Collections.emptySet()); + when(consumer.assignment()) + .thenReturn(INITIAL_ASSIGNMENT, INITIAL_ASSIGNMENT, INITIAL_ASSIGNMENT) + .thenReturn(newAssignment, newAssignment, newAssignment) + .thenReturn(Collections.singleton(TOPIC_PARTITION3), + Collections.singleton(TOPIC_PARTITION3), + Collections.singleton(TOPIC_PARTITION3)); + + INITIAL_ASSIGNMENT.forEach(tp -> when(consumer.position(tp)).thenReturn(FIRST_OFFSET)); + when(consumer.position(TOPIC_PARTITION3)).thenReturn(FIRST_OFFSET); + + when(consumer.poll(any(Duration.class))) + .thenAnswer((Answer>) invocation -> { + rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); return ConsumerRecords.empty(); - }); - EasyMock.expect(consumer.assignment()).andReturn(Collections.singleton(TOPIC_PARTITION)).times(2); - final Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - sinkTask.preCommit(offsets); - EasyMock.expectLastCall().andReturn(offsets); - consumer.commitSync(offsets); - EasyMock.expectLastCall(); - sinkTask.close(Collections.singleton(TOPIC_PARTITION)); - EasyMock.expectLastCall(); - sinkTask.put(Collections.emptyList()); - EasyMock.expectLastCall(); - - // Fourth poll; assignment changes to [TP2, TP3] - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { + }) + .thenAnswer(expectConsumerPoll(1)) + // Empty consumer poll (all partitions are paused) with rebalance; one new partition is assigned + .thenAnswer(invocation -> { rebalanceListener.getValue().onPartitionsRevoked(Collections.emptySet()); rebalanceListener.getValue().onPartitionsAssigned(Collections.singleton(TOPIC_PARTITION3)); return ConsumerRecords.empty(); + }) + .thenAnswer(expectConsumerPoll(0)) + // Non-empty consumer poll; all initially-assigned partitions are revoked in rebalance, and new partitions are allowed to resume + .thenAnswer(invocation -> { + ConsumerRecord newRecord = new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET, RAW_KEY, RAW_VALUE); + + rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); + rebalanceListener.getValue().onPartitionsAssigned(Collections.emptyList()); + return new ConsumerRecords<>(Collections.singletonMap(TOPIC_PARTITION3, Collections.singletonList(newRecord))); }); - EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))).times(2); - EasyMock.expect(consumer.position(TOPIC_PARTITION3)).andReturn(FIRST_OFFSET); - sinkTask.open(Collections.singleton(TOPIC_PARTITION3)); - EasyMock.expectLastCall(); - sinkTask.put(Collections.emptyList()); - EasyMock.expectLastCall(); - - // Fifth poll; an offset commit takes place - EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))).times(2); - final Map workerCurrentOffsets = new HashMap<>(); - workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - workerCurrentOffsets.put(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET)); - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andThrow(new ConnectException("Failed to flush")); + expectConversionAndTransformation(null, new RecordHeaders()); + + doNothing() + // If a retriable exception is thrown, we should redeliver the same batch, pausing the consumer in the meantime + .doThrow(new RetriableException("retry")) + .doThrow(new RetriableException("retry")) + .doThrow(new RetriableException("retry")) + .doNothing() + .when(sinkTask).put(any(Collection.class)); + + workerTask.iteration(); - consumer.seek(TOPIC_PARTITION2, FIRST_OFFSET); - EasyMock.expectLastCall(); - consumer.seek(TOPIC_PARTITION3, FIRST_OFFSET); - EasyMock.expectLastCall(); + // Pause + workerTask.iteration(); + verify(consumer).pause(INITIAL_ASSIGNMENT); - expectConsumerPoll(0); - sinkTask.put(EasyMock.eq(Collections.emptyList())); - EasyMock.expectLastCall(); + workerTask.iteration(); + verify(sinkTask).open(Collections.singleton(TOPIC_PARTITION3)); + // All partitions are re-paused in order to pause any newly-assigned partitions so that redelivery efforts can continue + verify(consumer).pause(newAssignment); - PowerMock.replayAll(); + workerTask.iteration(); + + final Map offsets = INITIAL_ASSIGNMENT.stream() + .collect(Collectors.toMap(Function.identity(), tp -> new OffsetAndMetadata(FIRST_OFFSET))); + when(sinkTask.preCommit(offsets)).thenReturn(offsets); + newAssignment = Collections.singleton(TOPIC_PARTITION3); + + workerTask.iteration(); + verify(sinkTask).close(INITIAL_ASSIGNMENT); + + // All partitions are resumed, as all previously paused-for-redelivery partitions were revoked + newAssignment.forEach(tp -> verify(consumer).resume(Collections.singleton(tp))); + } + + @Test + public void testErrorInRebalancePartitionLoss() { + RuntimeException exception = new RuntimeException("Revocation error"); + createTask(initialState); workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); - // First iteration--first call to poll, first consumer assignment + verifyInitializeTask(); + + expectPollInitialAssignment() + .thenAnswer((Answer>) invocation -> { + rebalanceListener.getValue().onPartitionsLost(INITIAL_ASSIGNMENT); + return ConsumerRecords.empty(); + }); + + doThrow(exception).when(sinkTask).close(INITIAL_ASSIGNMENT); + workerTask.iteration(); - // Second iteration--second call to poll, delivery of one record + verifyPollInitialAssignment(); + + RuntimeException thrownException = assertThrows(RuntimeException.class, () -> workerTask.iteration()); + assertEquals(exception, thrownException); + } + + @Test + public void testErrorInRebalancePartitionRevocation() { + RuntimeException exception = new RuntimeException("Revocation error"); + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectPollInitialAssignment() + .thenAnswer((Answer>) invocation -> { + rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); + return ConsumerRecords.empty(); + }); + + expectRebalanceRevocationError(exception); + workerTask.iteration(); - // Third iteration--third call to poll, partial consumer revocation + verifyPollInitialAssignment(); + + RuntimeException thrownException = assertThrows(RuntimeException.class, () -> workerTask.iteration()); + assertEquals(exception, thrownException); + } + + @Test + public void testErrorInRebalancePartitionAssignment() { + RuntimeException exception = new RuntimeException("Assignment error"); + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectPollInitialAssignment() + .thenAnswer((Answer>) invocation -> { + rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); + rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); + return ConsumerRecords.empty(); + }); + workerTask.iteration(); - // Fourth iteration--fourth call to poll, partial consumer assignment + verifyPollInitialAssignment(); + + expectRebalanceAssignmentError(exception); + + try { + RuntimeException thrownException = assertThrows(RuntimeException.class, () -> workerTask.iteration()); + assertEquals(exception, thrownException); + } finally { + verify(sinkTask).close(INITIAL_ASSIGNMENT); + } + } + + @Test + public void testPartialRevocationAndAssignment() { + createTask(initialState); + + when(consumer.assignment()) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(Collections.singleton(TOPIC_PARTITION2)) + .thenReturn(Collections.singleton(TOPIC_PARTITION2)) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(INITIAL_ASSIGNMENT); + + INITIAL_ASSIGNMENT.forEach(tp -> when(consumer.position(tp)).thenReturn(FIRST_OFFSET)); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + when(consumer.poll(any(Duration.class))) + .thenAnswer((Answer>) invocation -> { + rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); + return ConsumerRecords.empty(); + }) + .thenAnswer((Answer>) invocation -> { + rebalanceListener.getValue().onPartitionsRevoked(singleton(TOPIC_PARTITION)); + rebalanceListener.getValue().onPartitionsAssigned(Collections.emptySet()); + return ConsumerRecords.empty(); + }) + .thenAnswer((Answer>) invocation -> { + rebalanceListener.getValue().onPartitionsRevoked(Collections.emptySet()); + rebalanceListener.getValue().onPartitionsAssigned(singleton(TOPIC_PARTITION3)); + return ConsumerRecords.empty(); + }) + .thenAnswer((Answer>) invocation -> { + rebalanceListener.getValue().onPartitionsLost(singleton(TOPIC_PARTITION3)); + rebalanceListener.getValue().onPartitionsAssigned(singleton(TOPIC_PARTITION)); + return ConsumerRecords.empty(); + }); + + final Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); + when(sinkTask.preCommit(offsets)).thenReturn(offsets); + + when(consumer.position(TOPIC_PARTITION3)).thenReturn(FIRST_OFFSET); + + // First iteration--first call to poll, first consumer assignment workerTask.iteration(); - // Fifth iteration--task-requested offset commit with failure in SinkTask::preCommit - sinkTaskContext.getValue().requestCommit(); + verifyPollInitialAssignment(); + + // Second iteration--second call to poll, partial consumer revocation + workerTask.iteration(); + verify(sinkTask).close(singleton(TOPIC_PARTITION)); + verify(sinkTask, times(2)).put(Collections.emptyList()); + + // Third iteration--third call to poll, partial consumer assignment workerTask.iteration(); + verify(sinkTask).open(singleton(TOPIC_PARTITION3)); + verify(sinkTask, times(3)).put(Collections.emptyList()); - PowerMock.verifyAll(); + // Fourth iteration--fourth call to poll, one partition lost; can't commit offsets for it, one new partition assigned + workerTask.iteration(); + verify(sinkTask).close(singleton(TOPIC_PARTITION3)); + verify(sinkTask).open(singleton(TOPIC_PARTITION)); + verify(sinkTask, times(4)).put(Collections.emptyList()); } @Test - public void testWakeupInCommitSyncCausesRetry() throws Exception { + @SuppressWarnings("unchecked") + public void testPreCommitFailureAfterPartialRevocationAndAssignment() { createTask(initialState); + expectTaskGetTopic(); - expectInitializeTask(); - expectTaskGetTopic(true); - expectPollInitialAssignment(); + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); + when(consumer.assignment()) + .thenReturn(INITIAL_ASSIGNMENT, INITIAL_ASSIGNMENT) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2))) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2))) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2))) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))) + .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))); - final Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - sinkTask.preCommit(offsets); - EasyMock.expectLastCall().andReturn(offsets); + INITIAL_ASSIGNMENT.forEach(tp -> when(consumer.position(tp)).thenReturn(FIRST_OFFSET)); + when(consumer.position(TOPIC_PARTITION3)).thenReturn(FIRST_OFFSET); - // first one raises wakeup - consumer.commitSync(EasyMock.>anyObject()); - EasyMock.expectLastCall().andThrow(new WakeupException()); + // First poll; assignment is [TP1, TP2] + when(consumer.poll(any(Duration.class))) + .thenAnswer((Answer>) invocation -> { + rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); + return ConsumerRecords.empty(); + }) + // Second poll; a single record is delivered from TP1 + .thenAnswer(expectConsumerPoll(1)) + // Third poll; assignment changes to [TP2] + .thenAnswer(invocation -> { + rebalanceListener.getValue().onPartitionsRevoked(Collections.singleton(TOPIC_PARTITION)); + rebalanceListener.getValue().onPartitionsAssigned(Collections.emptySet()); + return ConsumerRecords.empty(); + }) + // Fourth poll; assignment changes to [TP2, TP3] + .thenAnswer(invocation -> { + rebalanceListener.getValue().onPartitionsRevoked(Collections.emptySet()); + rebalanceListener.getValue().onPartitionsAssigned(Collections.singleton(TOPIC_PARTITION3)); + return ConsumerRecords.empty(); + }) + // Fifth poll; an offset commit takes place + .thenAnswer(expectConsumerPoll(0)); - // we should retry and complete the commit - consumer.commitSync(EasyMock.>anyObject()); - EasyMock.expectLastCall(); + expectConversionAndTransformation(null, new RecordHeaders()); - sinkTask.close(INITIAL_ASSIGNMENT); - EasyMock.expectLastCall(); + // First iteration--first call to poll, first consumer assignment + workerTask.iteration(); + // Second iteration--second call to poll, delivery of one record + workerTask.iteration(); + // Third iteration--third call to poll, partial consumer revocation + final Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + when(sinkTask.preCommit(offsets)).thenReturn(offsets); + doNothing().when(consumer).commitSync(offsets); - INITIAL_ASSIGNMENT.forEach(tp -> EasyMock.expect(consumer.position(tp)).andReturn(FIRST_OFFSET)); + workerTask.iteration(); + verify(sinkTask).close(Collections.singleton(TOPIC_PARTITION)); + verify(sinkTask, times(2)).put(Collections.emptyList()); - sinkTask.open(INITIAL_ASSIGNMENT); - EasyMock.expectLastCall(); + // Fourth iteration--fourth call to poll, partial consumer assignment + workerTask.iteration(); - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(5); - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { - rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); - rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); - return ConsumerRecords.empty(); - }); + verify(sinkTask).open(Collections.singleton(TOPIC_PARTITION3)); - INITIAL_ASSIGNMENT.forEach(tp -> { - consumer.resume(Collections.singleton(tp)); - EasyMock.expectLastCall(); - }); + final Map workerCurrentOffsets = new HashMap<>(); + workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + workerCurrentOffsets.put(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET)); + when(sinkTask.preCommit(workerCurrentOffsets)).thenThrow(new ConnectException("Failed to flush")); - statusListener.onResume(taskId); - EasyMock.expectLastCall(); + // Fifth iteration--task-requested offset commit with failure in SinkTask::preCommit + sinkTaskContext.getValue().requestCommit(); + workerTask.iteration(); + + verify(consumer).seek(TOPIC_PARTITION2, FIRST_OFFSET); + verify(consumer).seek(TOPIC_PARTITION3, FIRST_OFFSET); + } - PowerMock.replayAll(); + @Test + public void testWakeupInCommitSyncCausesRetry() { + createTask(initialState); workerTask.initialize(TASK_CONFIG); time.sleep(30000L); workerTask.initializeAndStart(); time.sleep(30000L); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(invocation -> { + rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT); + rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); + return ConsumerRecords.empty(); + }); + expectConversionAndTransformation(null, new RecordHeaders()); workerTask.iteration(); // poll for initial assignment time.sleep(30000L); + + final Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + when(sinkTask.preCommit(offsets)).thenReturn(offsets); + + // first one raises wakeup + doThrow(new WakeupException()) + // and succeed the second time + .doNothing() + .when(consumer).commitSync(offsets); + workerTask.iteration(); // first record delivered + workerTask.iteration(); // now rebalance with the wakeup triggered time.sleep(30000L); + verify(sinkTask).close(INITIAL_ASSIGNMENT); + verify(sinkTask, times(2)).open(INITIAL_ASSIGNMENT); + + INITIAL_ASSIGNMENT.forEach(tp -> verify(consumer).resume(Collections.singleton(tp))); + + verify(statusListener).onResume(taskId); + assertSinkMetricValue("partition-count", 2); assertSinkMetricValue("sink-record-read-total", 1.0); assertSinkMetricValue("sink-record-send-total", 1.0); @@ -437,106 +815,69 @@ public void testWakeupInCommitSyncCausesRetry() throws Exception { assertTaskMetricValue("offset-commit-avg-time-ms", 0.0); assertTaskMetricValue("offset-commit-failure-percentage", 0.0); assertTaskMetricValue("offset-commit-success-percentage", 1.0); - - PowerMock.verifyAll(); } @Test - public void testWakeupNotThrownDuringShutdown() throws Exception { + @SuppressWarnings("unchecked") + public void testWakeupNotThrownDuringShutdown() { createTask(initialState); - expectInitializeTask(); - expectTaskGetTopic(true); - expectPollInitialAssignment(); - - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(() -> { - // stop the task during its second iteration - workerTask.stop(); - return new ConsumerRecords<>(Collections.emptyMap()); - }); - consumer.wakeup(); - EasyMock.expectLastCall(); - - sinkTask.put(EasyMock.eq(Collections.emptyList())); - EasyMock.expectLastCall(); - - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(1); + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(invocation -> { + // stop the task during its second iteration + workerTask.stop(); + return new ConsumerRecords<>(Collections.emptyMap()); + }); + expectConversionAndTransformation(null, new RecordHeaders()); final Map offsets = new HashMap<>(); offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - sinkTask.preCommit(offsets); - EasyMock.expectLastCall().andReturn(offsets); - - sinkTask.close(EasyMock.anyObject()); - PowerMock.expectLastCall(); + when(sinkTask.preCommit(offsets)).thenReturn(offsets); // fail the first time - consumer.commitSync(EasyMock.eq(offsets)); - EasyMock.expectLastCall().andThrow(new WakeupException()); - - // and succeed the second time - consumer.commitSync(EasyMock.eq(offsets)); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); + doThrow(new WakeupException()) + // and succeed the second time + .doNothing() + .when(consumer).commitSync(offsets); - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); workerTask.execute(); assertEquals(0, workerTask.commitFailures()); - - PowerMock.verifyAll(); + verify(consumer).wakeup(); + verify(sinkTask).close(any(Collection.class)); } @Test - public void testRequestCommit() throws Exception { + public void testRequestCommit() { createTask(initialState); - expectInitializeTask(); - expectTaskGetTopic(true); - expectPollInitialAssignment(); - - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - final Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - sinkTask.preCommit(offsets); - EasyMock.expectLastCall().andReturn(offsets); - - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(2); - - final Capture callback = EasyMock.newCapture(); - consumer.commitAsync(EasyMock.eq(offsets), EasyMock.capture(callback)); - EasyMock.expectLastCall().andAnswer(() -> { - callback.getValue().onComplete(offsets, null); - return null; - }); - - expectConsumerPoll(0); - sinkTask.put(Collections.emptyList()); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(expectConsumerPoll(0)); + expectConversionAndTransformation(null, new RecordHeaders()); // Initial assignment time.sleep(30000L); workerTask.iteration(); assertSinkMetricValue("partition-count", 2); + final Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + when(sinkTask.preCommit(offsets)).thenReturn(offsets); + // First record delivered workerTask.iteration(); assertSinkMetricValue("partition-count", 2); @@ -562,21 +903,27 @@ public void testRequestCommit() throws Exception { final long previousCommitValue = workerTask.getNextCommit(); sinkTaskContext.getValue().requestCommit(); assertTrue(sinkTaskContext.getValue().isCommitRequested()); - assertNotEquals(offsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); + assertNotEquals(offsets, workerTask.lastCommittedOffsets()); + + ArgumentCaptor callback = ArgumentCaptor.forClass(OffsetCommitCallback.class); time.sleep(10000L); workerTask.iteration(); // triggers the commit + verify(consumer).commitAsync(eq(offsets), callback.capture()); + callback.getValue().onComplete(offsets, null); time.sleep(10000L); + assertFalse(sinkTaskContext.getValue().isCommitRequested()); // should have been cleared - assertEquals(offsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); + assertEquals(offsets, workerTask.lastCommittedOffsets()); assertEquals(0, workerTask.commitFailures()); + // Assert the next commit time advances slightly, the amount it advances // is the normal commit time less the two sleeps since it started each // of those sleeps were 10 seconds. // KAFKA-8229 assertEquals("Should have only advanced by 40 seconds", - previousCommitValue + - (WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT - 10000L * 2), - workerTask.getNextCommit()); + previousCommitValue + + (WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT - 10000L * 2), + workerTask.getNextCommit()); assertSinkMetricValue("partition-count", 2); assertSinkMetricValue("sink-record-read-total", 1.0); @@ -596,30 +943,30 @@ public void testRequestCommit() throws Exception { assertTaskMetricValue("offset-commit-avg-time-ms", 0.0); assertTaskMetricValue("offset-commit-failure-percentage", 0.0); assertTaskMetricValue("offset-commit-success-percentage", 1.0); - - PowerMock.verifyAll(); } @Test - public void testPreCommit() throws Exception { + public void testPreCommit() { createTask(initialState); - expectInitializeTask(); - expectTaskGetTopic(true); + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); - // iter 1 - expectPollInitialAssignment(); + expectTaskGetTopic(); + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(2)) + .thenAnswer(expectConsumerPoll(0)); + expectConversionAndTransformation(null, new RecordHeaders()); - // iter 2 - expectConsumerPoll(2); - expectConversionAndTransformation(2); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); + workerTask.iteration(); // iter 1 -- initial assignment final Map workerStartingOffsets = new HashMap<>(); workerStartingOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); workerStartingOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + assertEquals(workerStartingOffsets, workerTask.currentOffsets()); + final Map workerCurrentOffsets = new HashMap<>(); workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 2)); workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); @@ -629,345 +976,291 @@ public void testPreCommit() throws Exception { taskOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET + 1)); // should be ignored because > current offset taskOffsets.put(new TopicPartition(TOPIC, 3), new OffsetAndMetadata(FIRST_OFFSET)); // should be ignored because this partition is not assigned + when(sinkTask.preCommit(workerCurrentOffsets)).thenReturn(taskOffsets); + + workerTask.iteration(); // iter 2 -- deliver 2 records + final Map committableOffsets = new HashMap<>(); committableOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); committableOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andReturn(taskOffsets); - // Expect extra invalid topic partition to be filtered, which causes the consumer assignment to be logged - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(2); - final Capture callback = EasyMock.newCapture(); - consumer.commitAsync(EasyMock.eq(committableOffsets), EasyMock.capture(callback)); - EasyMock.expectLastCall().andAnswer(() -> { - callback.getValue().onComplete(committableOffsets, null); - return null; - }); - expectConsumerPoll(0); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); + assertEquals(workerCurrentOffsets, workerTask.currentOffsets()); + assertEquals(workerStartingOffsets, workerTask.lastCommittedOffsets()); - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - workerTask.iteration(); // iter 1 -- initial assignment - - assertEquals(workerStartingOffsets, Whitebox.>getInternalState(workerTask, "currentOffsets")); - workerTask.iteration(); // iter 2 -- deliver 2 records - - assertEquals(workerCurrentOffsets, Whitebox.>getInternalState(workerTask, "currentOffsets")); - assertEquals(workerStartingOffsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); sinkTaskContext.getValue().requestCommit(); workerTask.iteration(); // iter 3 -- commit - assertEquals(committableOffsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); - PowerMock.verifyAll(); + // Expect extra invalid topic partition to be filtered, which causes the consumer assignment to be logged + ArgumentCaptor callback = ArgumentCaptor.forClass(OffsetCommitCallback.class); + verify(consumer).commitAsync(eq(committableOffsets), callback.capture()); + callback.getValue().onComplete(committableOffsets, null); + + assertEquals(committableOffsets, workerTask.lastCommittedOffsets()); } @Test - public void testPreCommitFailure() throws Exception { + public void testPreCommitFailure() { createTask(initialState); - expectInitializeTask(); - expectTaskGetTopic(true); - EasyMock.expect(consumer.assignment()).andStubReturn(INITIAL_ASSIGNMENT); + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); - // iter 1 - expectPollInitialAssignment(); + expectTaskGetTopic(); + expectPollInitialAssignment() + // Put one message through the task to get some offsets to commit + .thenAnswer(expectConsumerPoll(2)) + .thenAnswer(expectConsumerPoll(0)); - // iter 2 - expectConsumerPoll(2); - expectConversionAndTransformation(2); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); + expectConversionAndTransformation(null, new RecordHeaders()); + + workerTask.iteration(); // iter 1 -- initial assignment + + workerTask.iteration(); // iter 2 -- deliver 2 records // iter 3 final Map workerCurrentOffsets = new HashMap<>(); workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 2)); workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andThrow(new ConnectException("Failed to flush")); + when(sinkTask.preCommit(workerCurrentOffsets)).thenThrow(new ConnectException("Failed to flush")); - consumer.seek(TOPIC_PARTITION, FIRST_OFFSET); - EasyMock.expectLastCall(); - consumer.seek(TOPIC_PARTITION2, FIRST_OFFSET); - EasyMock.expectLastCall(); - - expectConsumerPoll(0); - sinkTask.put(EasyMock.eq(Collections.emptyList())); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - workerTask.iteration(); // iter 1 -- initial assignment - workerTask.iteration(); // iter 2 -- deliver 2 records sinkTaskContext.getValue().requestCommit(); workerTask.iteration(); // iter 3 -- commit - PowerMock.verifyAll(); + verify(consumer).seek(TOPIC_PARTITION, FIRST_OFFSET); + verify(consumer).seek(TOPIC_PARTITION2, FIRST_OFFSET); } @Test - public void testIgnoredCommit() throws Exception { + public void testIgnoredCommit() { createTask(initialState); - expectInitializeTask(); - expectTaskGetTopic(true); - - // iter 1 - expectPollInitialAssignment(); - - // iter 2 - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - final Map workerStartingOffsets = new HashMap<>(); - workerStartingOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); - workerStartingOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - - final Map workerCurrentOffsets = new HashMap<>(); - workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); - workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(2); + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); - // iter 3 - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andReturn(workerStartingOffsets); - // no actual consumer.commit() triggered - expectConsumerPoll(0); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); + expectTaskGetTopic(); + // iter 1 + expectPollInitialAssignment() + // iter 2 + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(expectConsumerPoll(0)); - PowerMock.replayAll(); + expectConversionAndTransformation(null, new RecordHeaders()); - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); workerTask.iteration(); // iter 1 -- initial assignment - assertEquals(workerStartingOffsets, Whitebox.>getInternalState(workerTask, "currentOffsets")); - assertEquals(workerStartingOffsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); + final Map workerStartingOffsets = new HashMap<>(); + workerStartingOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); + workerStartingOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + + assertEquals(workerStartingOffsets, workerTask.currentOffsets()); + assertEquals(workerStartingOffsets, workerTask.lastCommittedOffsets()); workerTask.iteration(); // iter 2 -- deliver 2 records + // iter 3 + final Map workerCurrentOffsets = new HashMap<>(); + workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + + when(sinkTask.preCommit(workerCurrentOffsets)).thenReturn(workerStartingOffsets); + sinkTaskContext.getValue().requestCommit(); + // no actual consumer.commit() triggered workerTask.iteration(); // iter 3 -- commit - - PowerMock.verifyAll(); } // Test that the commitTimeoutMs timestamp is correctly computed and checked in WorkerSinkTask.iteration() // when there is a long running commit in process. See KAFKA-4942 for more information. @Test - public void testLongRunningCommitWithoutTimeout() throws Exception { + public void testLongRunningCommitWithoutTimeout() { createTask(initialState); - expectInitializeTask(); - expectTaskGetTopic(true); - - // iter 1 - expectPollInitialAssignment(); + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); - // iter 2 - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); + expectTaskGetTopic(); + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1)) + // no actual consumer.commit() triggered + .thenAnswer(expectConsumerPoll(0)); + expectConversionAndTransformation(null, new RecordHeaders()); final Map workerStartingOffsets = new HashMap<>(); workerStartingOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); workerStartingOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + workerTask.iteration(); // iter 1 -- initial assignment + assertEquals(workerStartingOffsets, workerTask.currentOffsets()); + assertEquals(workerStartingOffsets, workerTask.lastCommittedOffsets()); + + time.sleep(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT); + workerTask.iteration(); // iter 2 -- deliver 2 records + final Map workerCurrentOffsets = new HashMap<>(); workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(2); - // iter 3 - note that we return the current offset to indicate they should be committed - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andReturn(workerCurrentOffsets); + when(sinkTask.preCommit(workerCurrentOffsets)).thenReturn(workerCurrentOffsets); - // We need to delay the result of trying to commit offsets to Kafka via the consumer.commitAsync - // method. We do this so that we can test that we do not erroneously mark a commit as timed out - // while it is still running and under time. To fake this for tests we have the commit run in a - // separate thread and wait for a latch which we control back in the main thread. - final ExecutorService executor = Executors.newSingleThreadExecutor(); - final CountDownLatch latch = new CountDownLatch(1); - - consumer.commitAsync(EasyMock.eq(workerCurrentOffsets), EasyMock.anyObject()); - EasyMock.expectLastCall().andAnswer(() -> { - // Grab the arguments passed to the consumer.commitAsync method - final Object[] args = EasyMock.getCurrentArguments(); - @SuppressWarnings("unchecked") - final Map offsets = (Map) args[0]; - final OffsetCommitCallback callback = (OffsetCommitCallback) args[1]; - - executor.execute(() -> { - try { - latch.await(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } + sinkTaskContext.getValue().requestCommit(); + workerTask.iteration(); // iter 3 -- commit in progress - callback.onComplete(offsets, null); - }); + // Make sure the "committing" flag didn't immediately get flipped back to false due to an incorrect timeout + assertTrue("Expected worker to be in the process of committing offsets", workerTask.isCommitting()); - return null; - }); + // Delay the result of trying to commit offsets to Kafka via the consumer.commitAsync method. + ArgumentCaptor offsetCommitCallbackArgumentCaptor = + ArgumentCaptor.forClass(OffsetCommitCallback.class); + verify(consumer).commitAsync(eq(workerCurrentOffsets), offsetCommitCallbackArgumentCaptor.capture()); - // no actual consumer.commit() triggered - expectConsumerPoll(0); + final OffsetCommitCallback callback = offsetCommitCallbackArgumentCaptor.getValue(); + callback.onComplete(workerCurrentOffsets, null); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); + assertEquals(workerCurrentOffsets, workerTask.currentOffsets()); + assertEquals(workerCurrentOffsets, workerTask.lastCommittedOffsets()); + assertFalse(workerTask.isCommitting()); + } - PowerMock.replayAll(); + @SuppressWarnings("unchecked") + @Test + public void testSinkTasksHandleCloseErrors() { + createTask(initialState); workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); - workerTask.iteration(); // iter 1 -- initial assignment + verifyInitializeTask(); - assertEquals(workerStartingOffsets, Whitebox.>getInternalState(workerTask, "currentOffsets")); - assertEquals(workerStartingOffsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); + expectTaskGetTopic(); + expectPollInitialAssignment() + // Put one message through the task to get some offsets to commit + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(expectConsumerPoll(1)); - time.sleep(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT); - workerTask.iteration(); // iter 2 -- deliver 2 records + expectConversionAndTransformation(null, new RecordHeaders()); - sinkTaskContext.getValue().requestCommit(); - workerTask.iteration(); // iter 3 -- commit in progress + doNothing() + .doAnswer(invocation -> { + workerTask.stop(); + return null; + }) + .when(sinkTask).put(anyList()); - // Make sure the "committing" flag didn't immediately get flipped back to false due to an incorrect timeout - assertTrue("Expected worker to be in the process of committing offsets", workerTask.isCommitting()); + Throwable closeException = new RuntimeException(); + when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap()); - // Let the async commit finish and wait for it to end - latch.countDown(); - executor.shutdown(); - executor.awaitTermination(30, TimeUnit.SECONDS); + // Throw another exception while closing the task's assignment + doThrow(closeException).when(sinkTask).close(any(Collection.class)); - assertEquals(workerCurrentOffsets, Whitebox.>getInternalState(workerTask, "currentOffsets")); - assertEquals(workerCurrentOffsets, Whitebox.>getInternalState(workerTask, "lastCommittedOffsets")); + RuntimeException thrownException = assertThrows(RuntimeException.class, () -> workerTask.execute()); + assertEquals(closeException, thrownException); - PowerMock.verifyAll(); + verify(consumer).wakeup(); } + @SuppressWarnings("unchecked") @Test - public void testSinkTasksHandleCloseErrors() throws Exception { + public void testSuppressCloseErrors() { createTask(initialState); - expectInitializeTask(); - expectTaskGetTopic(true); - expectPollInitialAssignment(); + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); - // Put one message through the task to get some offsets to commit - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - PowerMock.expectLastCall().andVoid(); - - // Stop the task during the next put - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - PowerMock.expectLastCall().andAnswer(() -> { - workerTask.stop(); - return null; - }); + expectTaskGetTopic(); + expectPollInitialAssignment() + // Put one message through the task to get some offsets to commit + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(expectConsumerPoll(1)); - consumer.wakeup(); - PowerMock.expectLastCall(); + expectConversionAndTransformation(null, new RecordHeaders()); - // Throw another exception while closing the task's assignment - EasyMock.expect(sinkTask.preCommit(EasyMock.anyObject())) - .andStubReturn(Collections.emptyMap()); + Throwable putException = new RuntimeException(); Throwable closeException = new RuntimeException(); - sinkTask.close(EasyMock.anyObject()); - PowerMock.expectLastCall().andThrow(closeException); - PowerMock.replayAll(); + doNothing() + // Throw an exception on the next put to trigger shutdown behavior + // This exception is the true "cause" of the failure + .doThrow(putException) + .when(sinkTask).put(anyList()); + + when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap()); + + // Throw another exception while closing the task's assignment + doThrow(closeException).when(sinkTask).close(any(Collection.class)); workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); - try { - workerTask.execute(); - fail("workerTask.execute should have thrown an exception"); - } catch (RuntimeException e) { - PowerMock.verifyAll(); - assertSame("Exception from close should propagate as-is", closeException, e); - } + + RuntimeException thrownException = assertThrows(ConnectException.class, () -> workerTask.execute()); + assertEquals("Exception from put should be the cause", putException, thrownException.getCause()); + assertTrue("Exception from close should be suppressed", thrownException.getSuppressed().length > 0); + assertEquals(closeException, thrownException.getSuppressed()[0]); } @Test - public void testSuppressCloseErrors() throws Exception { + public void testTaskCancelPreventsFinalOffsetCommit() { createTask(initialState); - expectInitializeTask(); - expectTaskGetTopic(true); - - expectPollInitialAssignment(); - // Put one message through the task to get some offsets to commit - expectConsumerPoll(1); - expectConversionAndTransformation(1); - sinkTask.put(EasyMock.anyObject()); - PowerMock.expectLastCall().andVoid(); - - // Throw an exception on the next put to trigger shutdown behavior - // This exception is the true "cause" of the failure - expectConsumerPoll(1); - expectConversionAndTransformation(1); - Throwable putException = new RuntimeException(); - sinkTask.put(EasyMock.anyObject()); - PowerMock.expectLastCall().andThrow(putException); + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + // Put one message through the task to get some offsets to commit + .thenAnswer(expectConsumerPoll(1)) + // the second put will return after the task is stopped and cancelled (asynchronously) + .thenAnswer(expectConsumerPoll(1)); + + expectConversionAndTransformation(null, new RecordHeaders()); + + doNothing() + .doNothing() + .doAnswer(invocation -> { + workerTask.stop(); + workerTask.cancel(); + return null; + }) + .when(sinkTask).put(anyList()); + + // task performs normal steps in advance of committing offsets + final Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 2)); + offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + when(sinkTask.preCommit(offsets)).thenReturn(offsets); - // Throw another exception while closing the task's assignment - EasyMock.expect(sinkTask.preCommit(EasyMock.anyObject())) - .andStubReturn(Collections.emptyMap()); - Throwable closeException = new RuntimeException(); - sinkTask.close(EasyMock.anyObject()); - PowerMock.expectLastCall().andThrow(closeException); + workerTask.execute(); - PowerMock.replayAll(); + // stop wakes up the consumer + verify(consumer).wakeup(); - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); - try { - workerTask.execute(); - fail("workerTask.execute should have thrown an exception"); - } catch (ConnectException e) { - PowerMock.verifyAll(); - assertSame("Exception from put should be the cause", putException, e.getCause()); - assertTrue("Exception from close should be suppressed", e.getSuppressed().length > 0); - assertSame(closeException, e.getSuppressed()[0]); - } + verify(sinkTask).close(any()); } // Verify that when commitAsync is called but the supplied callback is not called by the consumer before a // rebalance occurs, the async callback does not reset the last committed offset from the rebalance. // See KAFKA-5731 for more information. @Test - public void testCommitWithOutOfOrderCallback() throws Exception { + public void testCommitWithOutOfOrderCallback() { createTask(initialState); - expectInitializeTask(); - expectTaskGetTopic(true); + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); // iter 1 - expectPollInitialAssignment(); + Answer> consumerPollRebalance = invocation -> { + rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); + return ConsumerRecords.empty(); + }; // iter 2 - expectConsumerPoll(1); - expectConversionAndTransformation(4); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - final Map workerStartingOffsets = new HashMap<>(); - workerStartingOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET)); - workerStartingOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + expectTaskGetTopic(); + expectConversionAndTransformation(null, new RecordHeaders()); final Map workerCurrentOffsets = new HashMap<>(); workerCurrentOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); @@ -985,11 +1278,8 @@ public void testCommitWithOutOfOrderCallback() throws Exception { postRebalanceCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); postRebalanceCurrentOffsets.put(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET + 2)); - EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(originalPartitions)).times(2); - // iter 3 - note that we return the current offset to indicate they should be committed - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andReturn(workerCurrentOffsets); + when(sinkTask.preCommit(workerCurrentOffsets)).thenReturn(workerCurrentOffsets); // We need to delay the result of trying to commit offsets to Kafka via the consumer.commitAsync // method. We do this so that we can test that the callback is not called until after the rebalance @@ -998,111 +1288,94 @@ public void testCommitWithOutOfOrderCallback() throws Exception { final AtomicReference asyncCallbackRunner = new AtomicReference<>(); final AtomicBoolean asyncCallbackRan = new AtomicBoolean(); - consumer.commitAsync(EasyMock.eq(workerCurrentOffsets), EasyMock.anyObject()); - EasyMock.expectLastCall().andAnswer(() -> { - // Grab the arguments passed to the consumer.commitAsync method - final Object[] args = EasyMock.getCurrentArguments(); - @SuppressWarnings("unchecked") - final Map offsets = (Map) args[0]; - final OffsetCommitCallback callback = (OffsetCommitCallback) args[1]; + doAnswer(invocation -> { + final Map offsets = invocation.getArgument(0); + final OffsetCommitCallback callback = invocation.getArgument(1); asyncCallbackRunner.set(() -> { callback.onComplete(offsets, null); asyncCallbackRan.set(true); }); + return null; - }); + }).when(consumer).commitAsync(eq(workerCurrentOffsets), any(OffsetCommitCallback.class)); // Expect the next poll to discover and perform the rebalance, THEN complete the previous callback handler, // and then return one record for TP1 and one for TP3. final AtomicBoolean rebalanced = new AtomicBoolean(); - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { - // Rebalance always begins with revoking current partitions ... - rebalanceListener.getValue().onPartitionsRevoked(originalPartitions); - // Respond to the rebalance - Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, rebalanceOffsets.get(TOPIC_PARTITION).offset()); - offsets.put(TOPIC_PARTITION2, rebalanceOffsets.get(TOPIC_PARTITION2).offset()); - offsets.put(TOPIC_PARTITION3, rebalanceOffsets.get(TOPIC_PARTITION3).offset()); - sinkTaskContext.getValue().offset(offsets); - rebalanceListener.getValue().onPartitionsAssigned(rebalancedPartitions); - rebalanced.set(true); - - // Run the previous async commit handler - asyncCallbackRunner.get().run(); - - // And prep the two records to return - long timestamp = RecordBatch.NO_TIMESTAMP; - TimestampType timestampType = TimestampType.NO_TIMESTAMP_TYPE; - List> records = new ArrayList<>(); - records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, timestamp, timestampType, + Answer> consumerPollRebalanced = invocation -> { + // Rebalance always begins with revoking current partitions ... + rebalanceListener.getValue().onPartitionsRevoked(originalPartitions); + // Respond to the rebalance + Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, rebalanceOffsets.get(TOPIC_PARTITION).offset()); + offsets.put(TOPIC_PARTITION2, rebalanceOffsets.get(TOPIC_PARTITION2).offset()); + offsets.put(TOPIC_PARTITION3, rebalanceOffsets.get(TOPIC_PARTITION3).offset()); + sinkTaskContext.getValue().offset(offsets); + rebalanceListener.getValue().onPartitionsAssigned(rebalancedPartitions); + rebalanced.set(true); + + // Run the previous async commit handler + asyncCallbackRunner.get().run(); + + // And prep the two records to return + long timestamp = RecordBatch.NO_TIMESTAMP; + TimestampType timestampType = TimestampType.NO_TIMESTAMP_TYPE; + List> records = new ArrayList<>(); + records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, timestamp, timestampType, 0, 0, RAW_KEY, RAW_VALUE, new RecordHeaders(), Optional.empty())); - records.add(new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET + recordsReturnedTp3 + 1, timestamp, timestampType, + records.add(new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET + recordsReturnedTp3 + 1, timestamp, timestampType, 0, 0, RAW_KEY, RAW_VALUE, new RecordHeaders(), Optional.empty())); - recordsReturnedTp1 += 1; - recordsReturnedTp3 += 1; - return new ConsumerRecords<>(Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records)); - }); + recordsReturnedTp1 += 1; + recordsReturnedTp3 += 1; + return new ConsumerRecords<>(Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records)); + }; // onPartitionsRevoked - sinkTask.preCommit(workerCurrentOffsets); - EasyMock.expectLastCall().andReturn(workerCurrentOffsets); - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - sinkTask.close(new ArrayList<>(workerCurrentOffsets.keySet())); - EasyMock.expectLastCall(); - consumer.commitSync(workerCurrentOffsets); - EasyMock.expectLastCall(); + when(sinkTask.preCommit(workerCurrentOffsets)).thenReturn(workerCurrentOffsets); // onPartitionsAssigned - step 1 final long offsetTp1 = rebalanceOffsets.get(TOPIC_PARTITION).offset(); final long offsetTp2 = rebalanceOffsets.get(TOPIC_PARTITION2).offset(); final long offsetTp3 = rebalanceOffsets.get(TOPIC_PARTITION3).offset(); - EasyMock.expect(consumer.position(TOPIC_PARTITION)).andReturn(offsetTp1); - EasyMock.expect(consumer.position(TOPIC_PARTITION2)).andReturn(offsetTp2); - EasyMock.expect(consumer.position(TOPIC_PARTITION3)).andReturn(offsetTp3); - EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(rebalancedPartitions)).times(5); - - // onPartitionsAssigned - step 2 - sinkTask.open(EasyMock.eq(rebalancedPartitions)); - EasyMock.expectLastCall(); - - // onPartitionsAssigned - step 3 rewind - consumer.seek(TOPIC_PARTITION, offsetTp1); - EasyMock.expectLastCall(); - consumer.seek(TOPIC_PARTITION2, offsetTp2); - EasyMock.expectLastCall(); - consumer.seek(TOPIC_PARTITION3, offsetTp3); - EasyMock.expectLastCall(); // iter 4 - note that we return the current offset to indicate they should be committed - sinkTask.preCommit(postRebalanceCurrentOffsets); - EasyMock.expectLastCall().andReturn(postRebalanceCurrentOffsets); - - final Capture callback = EasyMock.newCapture(); - consumer.commitAsync(EasyMock.eq(postRebalanceCurrentOffsets), EasyMock.capture(callback)); - EasyMock.expectLastCall().andAnswer(() -> { - callback.getValue().onComplete(postRebalanceCurrentOffsets, null); - return null; - }); - - // no actual consumer.commit() triggered - expectConsumerPoll(1); - - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - - workerTask.initialize(TASK_CONFIG); - workerTask.initializeAndStart(); + when(sinkTask.preCommit(postRebalanceCurrentOffsets)).thenReturn(postRebalanceCurrentOffsets); + + // Setup mocks + when(consumer.assignment()) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(INITIAL_ASSIGNMENT) + .thenReturn(new HashSet<>(rebalancedPartitions)) + .thenReturn(new HashSet<>(rebalancedPartitions)) + .thenReturn(new HashSet<>(rebalancedPartitions)) + .thenReturn(new HashSet<>(rebalancedPartitions)) + .thenReturn(new HashSet<>(rebalancedPartitions)); + + when(consumer.position(TOPIC_PARTITION)) + .thenReturn(FIRST_OFFSET) + .thenReturn(offsetTp1); + + when(consumer.position(TOPIC_PARTITION2)) + .thenReturn(FIRST_OFFSET) + .thenReturn(offsetTp2); + + when(consumer.position(TOPIC_PARTITION3)) + .thenReturn(offsetTp3); + + when(consumer.poll(any(Duration.class))) + .thenAnswer(consumerPollRebalance) + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(consumerPollRebalanced) + .thenAnswer(expectConsumerPoll(1)); + + // Run the iterations workerTask.iteration(); // iter 1 -- initial assignment - assertEquals(workerStartingOffsets, Whitebox.getInternalState(workerTask, "currentOffsets")); - assertEquals(workerStartingOffsets, Whitebox.getInternalState(workerTask, "lastCommittedOffsets")); - time.sleep(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT); - workerTask.iteration(); // iter 2 -- deliver 2 records + workerTask.iteration(); // iter 2 -- deliver records sinkTaskContext.getValue().requestCommit(); workerTask.iteration(); // iter 3 -- commit in progress @@ -1130,16 +1403,32 @@ public void testCommitWithOutOfOrderCallback() throws Exception { assertTrue(rebalanced.get()); // Check that the offsets were not reset by the out-of-order async commit callback - assertEquals(postRebalanceCurrentOffsets, Whitebox.getInternalState(workerTask, "currentOffsets")); - assertEquals(rebalanceOffsets, Whitebox.getInternalState(workerTask, "lastCommittedOffsets")); + assertEquals(postRebalanceCurrentOffsets, workerTask.currentOffsets()); + assertEquals(rebalanceOffsets, workerTask.lastCommittedOffsets()); + + // onPartitionsRevoked + verify(sinkTask).close(new ArrayList<>(workerCurrentOffsets.keySet())); + verify(consumer).commitSync(anyMap()); + + // onPartitionsAssigned - step 2 + verify(sinkTask).open(rebalancedPartitions); + + // onPartitionsAssigned - step 3 rewind + verify(consumer).seek(TOPIC_PARTITION, offsetTp1); + verify(consumer).seek(TOPIC_PARTITION2, offsetTp2); + verify(consumer).seek(TOPIC_PARTITION3, offsetTp3); time.sleep(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT); sinkTaskContext.getValue().requestCommit(); workerTask.iteration(); // iter 4 -- commit in progress + final ArgumentCaptor callback = ArgumentCaptor.forClass(OffsetCommitCallback.class); + verify(consumer).commitAsync(eq(postRebalanceCurrentOffsets), callback.capture()); + callback.getValue().onComplete(postRebalanceCurrentOffsets, null); + // Check that the offsets were not reset by the out-of-order async commit callback - assertEquals(postRebalanceCurrentOffsets, Whitebox.getInternalState(workerTask, "currentOffsets")); - assertEquals(postRebalanceCurrentOffsets, Whitebox.getInternalState(workerTask, "lastCommittedOffsets")); + assertEquals(postRebalanceCurrentOffsets, workerTask.currentOffsets()); + assertEquals(postRebalanceCurrentOffsets, workerTask.lastCommittedOffsets()); assertSinkMetricValue("partition-count", 3); assertSinkMetricValue("sink-record-read-total", 4.0); @@ -1159,116 +1448,428 @@ public void testCommitWithOutOfOrderCallback() throws Exception { assertTaskMetricValue("offset-commit-avg-time-ms", 0.0); assertTaskMetricValue("offset-commit-failure-percentage", 0.0); assertTaskMetricValue("offset-commit-success-percentage", 1.0); + } + + @Test + public void testDeliveryWithMutatingTransform() { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectTaskGetTopic(); + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1)) + .thenAnswer(expectConsumerPoll(0)); + + expectConversionAndTransformation("newtopic_", new RecordHeaders()); + + workerTask.iteration(); // initial assignment + + workerTask.iteration(); // first record delivered + + final Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + offsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET)); + when(sinkTask.preCommit(offsets)).thenReturn(offsets); + + sinkTaskContext.getValue().requestCommit(); + assertTrue(sinkTaskContext.getValue().isCommitRequested()); + + assertNotEquals(offsets, workerTask.lastCommittedOffsets()); + workerTask.iteration(); // triggers the commit + + ArgumentCaptor callback = ArgumentCaptor.forClass(OffsetCommitCallback.class); + verify(consumer).commitAsync(eq(offsets), callback.capture()); + + callback.getValue().onComplete(offsets, null); - PowerMock.verifyAll(); + assertFalse(sinkTaskContext.getValue().isCommitRequested()); // should have been cleared + assertEquals(offsets, workerTask.lastCommittedOffsets()); + assertEquals(0, workerTask.commitFailures()); + assertEquals(1.0, metrics.currentMetricValueAsDouble(workerTask.taskMetricsGroup().metricGroup(), "batch-size-max"), 0.0001); } - private void expectInitializeTask() { - consumer.subscribe(EasyMock.eq(asList(TOPIC)), EasyMock.capture(rebalanceListener)); - PowerMock.expectLastCall(); + @Test + public void testMissingTimestampPropagation() { + createTask(initialState); + expectTaskGetTopic(); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1, RecordBatch.NO_TIMESTAMP, TimestampType.CREATE_TIME, new RecordHeaders())); - sinkTask.initialize(EasyMock.capture(sinkTaskContext)); - PowerMock.expectLastCall(); - sinkTask.start(TASK_PROPS); - PowerMock.expectLastCall(); + expectConversionAndTransformation(null, new RecordHeaders()); + + workerTask.iteration(); // iter 1 -- initial assignment + workerTask.iteration(); // iter 2 -- deliver 1 record + + @SuppressWarnings("unchecked") + ArgumentCaptor> records = ArgumentCaptor.forClass(Collection.class); + verify(sinkTask, times(2)).put(records.capture()); + + SinkRecord record = records.getValue().iterator().next(); + + // we expect null for missing timestamp, the sentinel value of Record.NO_TIMESTAMP is Kafka's API + assertNull(record.timestamp()); + assertEquals(TimestampType.CREATE_TIME, record.timestampType()); } - private void expectPollInitialAssignment() { - sinkTask.open(INITIAL_ASSIGNMENT); - EasyMock.expectLastCall(); + @Test + public void testTimestampPropagation() { + final Long timestamp = System.currentTimeMillis(); + final TimestampType timestampType = TimestampType.CREATE_TIME; - EasyMock.expect(consumer.assignment()).andReturn(INITIAL_ASSIGNMENT).times(2); + createTask(initialState); + expectTaskGetTopic(); - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(() -> { - rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); - return ConsumerRecords.empty(); - }); - INITIAL_ASSIGNMENT.forEach(tp -> EasyMock.expect(consumer.position(tp)).andReturn(FIRST_OFFSET)); + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1, timestamp, timestampType, new RecordHeaders())); + + expectConversionAndTransformation(null, new RecordHeaders()); + + workerTask.iteration(); // iter 1 -- initial assignment + workerTask.iteration(); // iter 2 -- deliver 1 record - sinkTask.put(Collections.emptyList()); - EasyMock.expectLastCall(); + @SuppressWarnings("unchecked") + ArgumentCaptor> records = ArgumentCaptor.forClass(Collection.class); + verify(sinkTask, times(2)).put(records.capture()); + + SinkRecord record = records.getValue().iterator().next(); + + assertEquals(timestamp, record.timestamp()); + assertEquals(timestampType, record.timestampType()); } - private void expectConsumerPoll(final int numMessages) { - expectConsumerPoll(numMessages, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, emptyHeaders()); + @Test + public void testTopicsRegex() { + Map props = new HashMap<>(TASK_PROPS); + props.remove("topics"); + props.put("topics.regex", "te.*"); + TaskConfig taskConfig = new TaskConfig(props); + + createTask(TargetState.PAUSED); + + workerTask.initialize(taskConfig); + workerTask.initializeAndStart(); + + ArgumentCaptor topicsRegex = ArgumentCaptor.forClass(Pattern.class); + + verify(consumer).subscribe(topicsRegex.capture(), rebalanceListener.capture()); + assertEquals("te.*", topicsRegex.getValue().pattern()); + verify(sinkTask).initialize(sinkTaskContext.capture()); + verify(sinkTask).start(props); + + expectPollInitialAssignment(); + + workerTask.iteration(); + time.sleep(10000L); + + verify(consumer).pause(INITIAL_ASSIGNMENT); } - private void expectConsumerPoll(final int numMessages, final long timestamp, final TimestampType timestampType, Headers headers) { - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - () -> { - List> records = new ArrayList<>(); - for (int i = 0; i < numMessages; i++) - records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + i, timestamp, timestampType, - 0, 0, RAW_KEY, RAW_VALUE, headers, Optional.empty())); - recordsReturnedTp1 += numMessages; - return new ConsumerRecords<>( - numMessages > 0 ? - Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records) : - Collections.emptyMap() - ); - }); + @Test + public void testMetricsGroup() { + SinkTaskMetricsGroup group = new SinkTaskMetricsGroup(taskId, metrics); + SinkTaskMetricsGroup group1 = new SinkTaskMetricsGroup(taskId1, metrics); + for (int i = 0; i != 10; ++i) { + group.recordRead(1); + group.recordSend(2); + group.recordPut(3); + group.recordPartitionCount(4); + group.recordOffsetSequenceNumber(5); + } + Map committedOffsets = new HashMap<>(); + committedOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 1)); + group.recordCommittedOffsets(committedOffsets); + Map consumedOffsets = new HashMap<>(); + consumedOffsets.put(TOPIC_PARTITION, new OffsetAndMetadata(FIRST_OFFSET + 10)); + group.recordConsumedOffsets(consumedOffsets); + + for (int i = 0; i != 20; ++i) { + group1.recordRead(1); + group1.recordSend(2); + group1.recordPut(30); + group1.recordPartitionCount(40); + group1.recordOffsetSequenceNumber(50); + } + committedOffsets = new HashMap<>(); + committedOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET + 2)); + committedOffsets.put(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET + 3)); + group1.recordCommittedOffsets(committedOffsets); + consumedOffsets = new HashMap<>(); + consumedOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET + 20)); + consumedOffsets.put(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET + 30)); + group1.recordConsumedOffsets(consumedOffsets); + + assertEquals(0.333, metrics.currentMetricValueAsDouble(group.metricGroup(), "sink-record-read-rate"), 0.001d); + assertEquals(0.667, metrics.currentMetricValueAsDouble(group.metricGroup(), "sink-record-send-rate"), 0.001d); + assertEquals(9, metrics.currentMetricValueAsDouble(group.metricGroup(), "sink-record-active-count"), 0.001d); + assertEquals(4, metrics.currentMetricValueAsDouble(group.metricGroup(), "partition-count"), 0.001d); + assertEquals(5, metrics.currentMetricValueAsDouble(group.metricGroup(), "offset-commit-seq-no"), 0.001d); + assertEquals(3, metrics.currentMetricValueAsDouble(group.metricGroup(), "put-batch-max-time-ms"), 0.001d); + + // Close the group + group.close(); + + for (MetricName metricName : group.metricGroup().metrics().metrics().keySet()) { + // Metrics for this group should no longer exist + assertFalse(group.metricGroup().groupId().includes(metricName)); + } + // Sensors for this group should no longer exist + assertNull(group.metricGroup().metrics().getSensor("source-record-poll")); + assertNull(group.metricGroup().metrics().getSensor("source-record-write")); + assertNull(group.metricGroup().metrics().getSensor("poll-batch-time")); + + assertEquals(0.667, metrics.currentMetricValueAsDouble(group1.metricGroup(), "sink-record-read-rate"), 0.001d); + assertEquals(1.333, metrics.currentMetricValueAsDouble(group1.metricGroup(), "sink-record-send-rate"), 0.001d); + assertEquals(45, metrics.currentMetricValueAsDouble(group1.metricGroup(), "sink-record-active-count"), 0.001d); + assertEquals(40, metrics.currentMetricValueAsDouble(group1.metricGroup(), "partition-count"), 0.001d); + assertEquals(50, metrics.currentMetricValueAsDouble(group1.metricGroup(), "offset-commit-seq-no"), 0.001d); + assertEquals(30, metrics.currentMetricValueAsDouble(group1.metricGroup(), "put-batch-max-time-ms"), 0.001d); + } + + @Test + public void testHeaders() { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + Headers headers = new RecordHeaders(); + headers.add("header_key", "header_value".getBytes()); + + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1, headers)); + + expectConversionAndTransformation(null, headers); + + workerTask.iteration(); // iter 1 -- initial assignment + workerTask.iteration(); // iter 2 -- deliver 1 record + + @SuppressWarnings("unchecked") + ArgumentCaptor> recordCapture = ArgumentCaptor.forClass(Collection.class); + verify(sinkTask, times(2)).put(recordCapture.capture()); + + assertEquals(1, recordCapture.getValue().size()); + SinkRecord record = recordCapture.getValue().iterator().next(); + + assertEquals("header_value", record.headers().lastWithName("header_key").value()); + } + + @Test + public void testHeadersWithCustomConverter() { + StringConverter stringConverter = new StringConverter(); + SampleConverterWithHeaders testConverter = new SampleConverterWithHeaders(); + + createTask(initialState, stringConverter, testConverter, stringConverter); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + String keyA = "a"; + String valueA = "Árvíztűrő tükörfúrógép"; + Headers headersA = new RecordHeaders(); + String encodingA = "latin2"; + headersA.add("encoding", encodingA.getBytes()); + + String keyB = "b"; + String valueB = "Тестовое сообщение"; + Headers headersB = new RecordHeaders(); + String encodingB = "koi8_r"; + headersB.add("encoding", encodingB.getBytes()); + + expectPollInitialAssignment() + .thenAnswer((Answer>) invocation -> { + List> records = Arrays.asList( + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, + 0, 0, keyA.getBytes(), valueA.getBytes(encodingA), headersA, Optional.empty()), + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 2, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, + 0, 0, keyB.getBytes(), valueB.getBytes(encodingB), headersB, Optional.empty()) + ); + return new ConsumerRecords<>(Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records)); + }); + + expectTransformation(null); + + workerTask.iteration(); // iter 1 -- initial assignment + workerTask.iteration(); // iter 2 -- deliver records + + @SuppressWarnings("unchecked") + ArgumentCaptor> records = ArgumentCaptor.forClass(Collection.class); + verify(sinkTask, times(2)).put(records.capture()); + + Iterator iterator = records.getValue().iterator(); + + SinkRecord recordA = iterator.next(); + assertEquals(keyA, recordA.key()); + assertEquals(valueA, recordA.value()); + + SinkRecord recordB = iterator.next(); + assertEquals(keyB, recordB.key()); + assertEquals(valueB, recordB.value()); + } + + @Test + public void testOriginalTopicWithTopicMutatingTransformations() { + createTask(initialState); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + verifyInitializeTask(); + + expectPollInitialAssignment() + .thenAnswer(expectConsumerPoll(1)); + + expectConversionAndTransformation("newtopic_", new RecordHeaders()); + + workerTask.iteration(); // initial assignment + workerTask.iteration(); // first record delivered + + @SuppressWarnings("unchecked") + ArgumentCaptor> recordCapture = ArgumentCaptor.forClass(Collection.class); + verify(sinkTask, times(2)).put(recordCapture.capture()); + + assertEquals(1, recordCapture.getValue().size()); + SinkRecord record = recordCapture.getValue().iterator().next(); + assertEquals(TOPIC, record.originalTopic()); + assertEquals("newtopic_" + TOPIC, record.topic()); + } + + @Test + public void testPartitionCountInCaseOfPartitionRevocation() { + MockConsumer mockConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + // Setting up Worker Sink Task to check metrics + workerTask = new WorkerSinkTask( + taskId, sinkTask, statusListener, TargetState.PAUSED, workerConfig, ClusterConfigState.EMPTY, metrics, + keyConverter, valueConverter, errorHandlingMetrics, headerConverter, + transformationChain, mockConsumer, pluginLoader, time, + RetryWithToleranceOperatorTest.noopOperator(), null, statusBackingStore, Collections::emptyList); + mockConsumer.updateBeginningOffsets( + new HashMap() {{ + put(TOPIC_PARTITION, 0L); + put(TOPIC_PARTITION2, 0L); + }} + ); + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + // Initial Re-balance to assign INITIAL_ASSIGNMENT which is "TOPIC_PARTITION" and "TOPIC_PARTITION2" + mockConsumer.rebalance(INITIAL_ASSIGNMENT); + assertSinkMetricValue("partition-count", 2); + // Revoked "TOPIC_PARTITION" and second re-balance with "TOPIC_PARTITION2" + mockConsumer.rebalance(Collections.singleton(TOPIC_PARTITION2)); + assertSinkMetricValue("partition-count", 1); + // Closing the Worker Sink Task which will update the partition count as 0. + workerTask.close(); + assertSinkMetricValue("partition-count", 0); + } + + private void expectRebalanceRevocationError(RuntimeException e) { + when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap()); + doThrow(e).when(sinkTask).close(INITIAL_ASSIGNMENT); + } + + private void expectRebalanceAssignmentError(RuntimeException e) { + when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap()); + when(consumer.position(TOPIC_PARTITION)).thenReturn(FIRST_OFFSET); + when(consumer.position(TOPIC_PARTITION2)).thenReturn(FIRST_OFFSET); + + doThrow(e).when(sinkTask).open(INITIAL_ASSIGNMENT); + } + + private void verifyInitializeTask() { + verify(consumer).subscribe(eq(asList(TOPIC)), rebalanceListener.capture()); + verify(sinkTask).initialize(sinkTaskContext.capture()); + verify(sinkTask).start(TASK_PROPS); + } + + private OngoingStubbing> expectPollInitialAssignment() { + when(consumer.assignment()).thenReturn(INITIAL_ASSIGNMENT); + INITIAL_ASSIGNMENT.forEach(tp -> when(consumer.position(tp)).thenReturn(FIRST_OFFSET)); + + return when(consumer.poll(any(Duration.class))).thenAnswer( + invocation -> { + rebalanceListener.getValue().onPartitionsAssigned(INITIAL_ASSIGNMENT); + return ConsumerRecords.empty(); + } + ); + } + + private void verifyPollInitialAssignment() { + verify(sinkTask).open(INITIAL_ASSIGNMENT); + verify(consumer, atLeastOnce()).assignment(); + verify(sinkTask).put(Collections.emptyList()); + } + + private Answer> expectConsumerPoll(final int numMessages) { + return expectConsumerPoll(numMessages, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, new RecordHeaders()); } - private void expectConversionAndTransformation(final int numMessages) { - expectConversionAndTransformation(numMessages, null); + private Answer> expectConsumerPoll(final int numMessages, Headers headers) { + return expectConsumerPoll(numMessages, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, headers); } - private void expectConversionAndTransformation(final int numMessages, final String topicPrefix) { - expectConversionAndTransformation(numMessages, topicPrefix, emptyHeaders()); + private Answer> expectConsumerPoll(final int numMessages, final long timestamp, final TimestampType timestampType, Headers headers) { + return invocation -> { + List> records = new ArrayList<>(); + for (int i = 0; i < numMessages; i++) + records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + i, timestamp, timestampType, + 0, 0, RAW_KEY, RAW_VALUE, headers, Optional.empty())); + recordsReturnedTp1 += numMessages; + return new ConsumerRecords<>( + numMessages > 0 ? + Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records) + : Collections.emptyMap() + ); + }; } - private void expectConversionAndTransformation(final int numMessages, final String topicPrefix, final Headers headers) { - EasyMock.expect(keyConverter.toConnectData(TOPIC, headers, RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)).times(numMessages); - EasyMock.expect(valueConverter.toConnectData(TOPIC, headers, RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)).times(numMessages); + private void expectConversionAndTransformation(final String topicPrefix, final Headers headers) { + when(keyConverter.toConnectData(TOPIC, headers, RAW_KEY)).thenReturn(new SchemaAndValue(KEY_SCHEMA, KEY)); + when(valueConverter.toConnectData(TOPIC, headers, RAW_VALUE)).thenReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)); for (Header header : headers) { - EasyMock.expect(headerConverter.toConnectHeader(TOPIC, header.key(), header.value())).andReturn(new SchemaAndValue(VALUE_SCHEMA, new String(header.value()))).times(1); + when(headerConverter.toConnectHeader(TOPIC, header.key(), header.value())).thenReturn(new SchemaAndValue(VALUE_SCHEMA, new String(header.value()))); } - expectTransformation(numMessages, topicPrefix); + expectTransformation(topicPrefix); } - private void expectTransformation(final int numMessages, final String topicPrefix) { - final Capture recordCapture = EasyMock.newCapture(); - EasyMock.expect(transformationChain.apply(EasyMock.anyObject(), EasyMock.capture(recordCapture))) - .andAnswer(() -> { - SinkRecord origRecord = recordCapture.getValue(); + @SuppressWarnings("unchecked") + private void expectTransformation(final String topicPrefix) { + when(transformationChain.apply(any(ProcessingContext.class), any(SinkRecord.class))).thenAnswer((Answer) + invocation -> { + SinkRecord origRecord = invocation.getArgument(1); return topicPrefix != null && !topicPrefix.isEmpty() - ? origRecord.newRecord( - topicPrefix + origRecord.topic(), - origRecord.kafkaPartition(), - origRecord.keySchema(), - origRecord.key(), - origRecord.valueSchema(), - origRecord.value(), - origRecord.timestamp(), - origRecord.headers() - ) - : origRecord; - }).times(numMessages); + ? origRecord.newRecord( + topicPrefix + origRecord.topic(), + origRecord.kafkaPartition(), + origRecord.keySchema(), + origRecord.key(), + origRecord.valueSchema(), + origRecord.value(), + origRecord.timestamp(), + origRecord.headers() + ) : origRecord; + }); } - private void expectTaskGetTopic(boolean anyTimes) { - final Capture connectorCapture = EasyMock.newCapture(); - final Capture topicCapture = EasyMock.newCapture(); - IExpectationSetters expect = EasyMock.expect(statusBackingStore.getTopic( - EasyMock.capture(connectorCapture), - EasyMock.capture(topicCapture))); - if (anyTimes) { - expect.andStubAnswer(() -> new TopicStatus( - topicCapture.getValue(), - new ConnectorTaskId(connectorCapture.getValue(), 0), - Time.SYSTEM.milliseconds())); - } else { - expect.andAnswer(() -> new TopicStatus( - topicCapture.getValue(), - new ConnectorTaskId(connectorCapture.getValue(), 0), - Time.SYSTEM.milliseconds())); - } - if (connectorCapture.hasCaptured() && topicCapture.hasCaptured()) { - assertEquals("job", connectorCapture.getValue()); - assertEquals(TOPIC, topicCapture.getValue()); - } + private void expectTaskGetTopic() { + when(statusBackingStore.getTopic(anyString(), anyString())).thenAnswer((Answer) invocation -> { + String connector = invocation.getArgument(0, String.class); + String topic = invocation.getArgument(1, String.class); + return new TopicStatus(topic, new ConnectorTaskId(connector, 0), Time.SYSTEM.milliseconds()); + }); } private void assertSinkMetricValue(String name, double expected) { @@ -1288,11 +1889,4 @@ private void assertTaskMetricValue(String name, String expected) { String measured = metrics.currentMetricValueAsString(taskGroup, name); assertEquals(expected, measured); } - - private RecordHeaders emptyHeaders() { - return new RecordHeaders(); - } - - private abstract static class TestSinkTask extends SinkTask { - } -} +} \ No newline at end of file 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 c48abb0130..c09fa093fb 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 @@ -154,6 +154,7 @@ import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyCollection; import static org.mockito.ArgumentMatchers.anyLong; @@ -2123,7 +2124,7 @@ public void testNormalizeSourceConnectorOffsets() throws Exception { Collections.singletonMap("position", 20) ); - assertTrue(offsets.values().iterator().next().get("position") instanceof Integer); + assertInstanceOf(Integer.class, offsets.values().iterator().next().get("position")); mockInternalConverters(); @@ -2135,7 +2136,7 @@ public void testNormalizeSourceConnectorOffsets() throws Exception { assertEquals(1, normalizedOffsets.size()); // The integer value 20 gets deserialized as a long value by the JsonConverter - assertTrue(normalizedOffsets.values().iterator().next().get("position") instanceof Long); + assertInstanceOf(Long.class, normalizedOffsets.values().iterator().next().get("position")); } @Test 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 6c5ce0fdb5..e680f41d20 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 @@ -126,6 +126,7 @@ import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.mockito.AdditionalMatchers.leq; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; @@ -868,7 +869,7 @@ public void testCreateConnectorFailedValidation() throws Exception { ArgumentCaptor error = ArgumentCaptor.forClass(Throwable.class); verify(putConnectorCallback).onCompletion(error.capture(), isNull()); - assertTrue(error.getValue() instanceof BadRequestException); + assertInstanceOf(BadRequestException.class, error.getValue()); verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore, putConnectorCallback); assertEquals( @@ -1174,7 +1175,7 @@ public void testRestartConnectorAndTasksUnknownConnector() throws Exception { herder.restartConnectorAndTasks(restartRequest, callback); herder.tick(); ExecutionException ee = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); - assertTrue(ee.getCause() instanceof NotFoundException); + assertInstanceOf(NotFoundException.class, ee.getCause()); assertTrue(ee.getMessage().contains("Unknown connector:")); verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); @@ -1200,7 +1201,7 @@ public void testRestartConnectorAndTasksNotLeader() throws Exception { herder.restartConnectorAndTasks(restartRequest, callback); herder.tick(); ExecutionException ee = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); - assertTrue(ee.getCause() instanceof NotLeaderException); + assertInstanceOf(NotLeaderException.class, ee.getCause()); verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @@ -1228,7 +1229,7 @@ public void testRestartConnectorAndTasksUnknownStatus() throws Exception { herder.restartConnectorAndTasks(restartRequest, callback); herder.tick(); ExecutionException ee = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); - assertTrue(ee.getCause() instanceof NotFoundException); + assertInstanceOf(NotFoundException.class, ee.getCause()); assertTrue(ee.getMessage().contains("Status for connector")); verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); @@ -1837,7 +1838,7 @@ public void testStopConnectorNotLeader() throws Exception { ExecutionException.class, () -> cb.get(0, TimeUnit.SECONDS) ); - assertTrue(e.getCause() instanceof NotLeaderException); + assertInstanceOf(NotLeaderException.class, e.getCause()); verifyNoMoreInteractions(worker, member, configBackingStore, statusBackingStore); } @@ -2470,7 +2471,7 @@ public void testPutTaskConfigsMissingRequiredSignature() { ArgumentCaptor errorCapture = ArgumentCaptor.forClass(Throwable.class); verify(taskConfigCb).onCompletion(errorCapture.capture(), isNull()); - assertTrue(errorCapture.getValue() instanceof BadRequestException); + assertInstanceOf(BadRequestException.class, errorCapture.getValue()); verifyNoMoreInteractions(member, taskConfigCb); } @@ -2487,7 +2488,7 @@ public void testPutTaskConfigsDisallowedSignatureAlgorithm() { ArgumentCaptor errorCapture = ArgumentCaptor.forClass(Throwable.class); verify(taskConfigCb).onCompletion(errorCapture.capture(), isNull()); - assertTrue(errorCapture.getValue() instanceof BadRequestException); + assertInstanceOf(BadRequestException.class, errorCapture.getValue()); verifyNoMoreInteractions(member, taskConfigCb); } @@ -2513,7 +2514,7 @@ public void testPutTaskConfigsInvalidSignature() { ArgumentCaptor errorCapture = ArgumentCaptor.forClass(Throwable.class); verify(taskConfigCb).onCompletion(errorCapture.capture(), isNull()); - assertTrue(errorCapture.getValue() instanceof ConnectRestException); + assertInstanceOf(ConnectRestException.class, errorCapture.getValue()); assertEquals(FORBIDDEN.getStatusCode(), ((ConnectRestException) errorCapture.getValue()).statusCode()); verifyNoMoreInteractions(member, taskConfigCb); @@ -2531,7 +2532,7 @@ public void putTaskConfigsWorkerStillStarting() { ArgumentCaptor errorCapture = ArgumentCaptor.forClass(Throwable.class); verify(taskConfigCb).onCompletion(errorCapture.capture(), isNull()); - assertTrue(errorCapture.getValue() instanceof ConnectRestException); + assertInstanceOf(ConnectRestException.class, errorCapture.getValue()); assertEquals(SERVICE_UNAVAILABLE.getStatusCode(), ((ConnectRestException) errorCapture.getValue()).statusCode()); verifyNoMoreInteractions(member, taskConfigCb); @@ -2667,7 +2668,7 @@ public void testFenceZombiesInvalidSignature() { ArgumentCaptor errorCapture = ArgumentCaptor.forClass(Throwable.class); verify(taskConfigCb).onCompletion(errorCapture.capture(), isNull()); - assertTrue(errorCapture.getValue() instanceof ConnectRestException); + assertInstanceOf(ConnectRestException.class, errorCapture.getValue()); assertEquals(FORBIDDEN.getStatusCode(), ((ConnectRestException) errorCapture.getValue()).statusCode()); verifyNoMoreInteractions(member); @@ -2723,7 +2724,7 @@ private void testTaskRequestedZombieFencingForwardingToLeader(boolean succeed) t if (!succeed) { ExecutionException fencingException = assertThrows(ExecutionException.class, () -> fencing.get(10, TimeUnit.SECONDS)); - assertTrue(fencingException.getCause() instanceof ConnectException); + assertInstanceOf(ConnectException.class, fencingException.getCause()); } else { fencing.get(10, TimeUnit.SECONDS); } @@ -2959,7 +2960,7 @@ public void testExternalZombieFencingRequestAsynchronousFailure() throws Excepti herderFencingCallbacks.getAllValues().forEach(cb -> cb.accept(null, fencingException)); ExecutionException exception = assertThrows(ExecutionException.class, () -> fencing.get(10, TimeUnit.SECONDS)); - assertTrue(exception.getCause() instanceof ConnectException); + assertInstanceOf(ConnectException.class, exception.getCause()); stopBackgroundHerder(); @@ -3657,7 +3658,7 @@ public void testModifyConnectorOffsetsUnknownConnector() throws Exception { herder.modifyConnectorOffsets("connector-does-not-exist", new HashMap<>(), callback); herder.tick(); ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); - assertTrue(e.getCause() instanceof NotFoundException); + assertInstanceOf(NotFoundException.class, e.getCause()); } @Test @@ -3675,7 +3676,7 @@ public void testModifyOffsetsConnectorNotInStoppedState() throws Exception { herder.modifyConnectorOffsets(CONN1, null, callback); herder.tick(); ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); - assertTrue(e.getCause() instanceof BadRequestException); + assertInstanceOf(BadRequestException.class, e.getCause()); } @Test @@ -3693,7 +3694,7 @@ public void testModifyOffsetsNotLeader() throws Exception { herder.modifyConnectorOffsets(CONN1, new HashMap<>(), callback); herder.tick(); ExecutionException e = assertThrows(ExecutionException.class, () -> callback.get(1000L, TimeUnit.MILLISECONDS)); - assertTrue(e.getCause() instanceof NotLeaderException); + assertInstanceOf(NotLeaderException.class, e.getCause()); } @Test 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 cd7bd90c92..216e8c86c1 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 @@ -59,6 +59,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -220,7 +221,7 @@ public void testLogReportAndReturnFuture() { "partition=5, offset=100}.", msg); Future future = logReporter.report(context); - assertTrue(future instanceof CompletableFuture); + assertInstanceOf(CompletableFuture.class, future); } @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 72fe97c2cb..4b60223cbb 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 @@ -71,6 +71,7 @@ import static org.junit.Assert.assertSame; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; public class PluginsTest { @@ -151,7 +152,7 @@ public void shouldInstantiateAndConfigureExplicitlySetHeaderConverterWithCurrent WorkerConfig.HEADER_CONVERTER_CLASS_CONFIG, ClassLoaderUsage.CURRENT_CLASSLOADER); assertNotNull(headerConverter); - assertTrue(headerConverter instanceof TestHeaderConverter); + assertInstanceOf(TestHeaderConverter.class, headerConverter); this.headerConverter = (TestHeaderConverter) headerConverter; // Validate extra configs got passed through to overridden converters @@ -162,7 +163,7 @@ public void shouldInstantiateAndConfigureExplicitlySetHeaderConverterWithCurrent WorkerConfig.HEADER_CONVERTER_CLASS_CONFIG, ClassLoaderUsage.PLUGINS); assertNotNull(headerConverter); - assertTrue(headerConverter instanceof TestHeaderConverter); + assertInstanceOf(TestHeaderConverter.class, headerConverter); this.headerConverter = (TestHeaderConverter) headerConverter; // Validate extra configs got passed through to overridden converters @@ -207,7 +208,7 @@ public void shouldInstantiateAndConfigureDefaultHeaderConverter() { WorkerConfig.HEADER_CONVERTER_CLASS_CONFIG, ClassLoaderUsage.PLUGINS); assertNotNull(headerConverter); - assertTrue(headerConverter instanceof SimpleHeaderConverter); + assertInstanceOf(SimpleHeaderConverter.class, headerConverter); } @Test @@ -657,13 +658,6 @@ public static void assertPluginClassLoaderAlwaysActive(Object plugin) { } } - 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/rest/util/SSLUtilsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/util/SSLUtilsTest.java index 8d486bbb9a..8f5c3235e7 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/util/SSLUtilsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/util/SSLUtilsTest.java @@ -26,7 +26,6 @@ import java.util.HashMap; import java.util.Map; -@SuppressWarnings("deprecation") public class SSLUtilsTest { @Test @@ -63,7 +62,7 @@ public void testCreateServerSideSslContextFactory() { configMap.put("ssl.trustmanager.algorithm", "PKIX"); RestServerConfig config = RestServerConfig.forPublic(null, configMap); - SslContextFactory ssl = SSLUtils.createServerSideSslContextFactory(config); + SslContextFactory.Server ssl = SSLUtils.createServerSideSslContextFactory(config); Assert.assertEquals("file:///path/to/keystore", ssl.getKeyStorePath()); Assert.assertEquals("file:///path/to/truststore", ssl.getTrustStorePath()); @@ -101,15 +100,13 @@ public void testCreateClientSideSslContextFactory() { configMap.put("ssl.trustmanager.algorithm", "PKIX"); RestServerConfig config = RestServerConfig.forPublic(null, configMap); - SslContextFactory ssl = SSLUtils.createClientSideSslContextFactory(config); + SslContextFactory.Client ssl = SSLUtils.createClientSideSslContextFactory(config); Assert.assertEquals("file:///path/to/keystore", ssl.getKeyStorePath()); Assert.assertEquals("file:///path/to/truststore", ssl.getTrustStorePath()); Assert.assertEquals("SunJSSE", ssl.getProvider()); Assert.assertArrayEquals(new String[] {"SSL_RSA_WITH_RC4_128_SHA", "SSL_RSA_WITH_RC4_128_MD5"}, ssl.getIncludeCipherSuites()); Assert.assertEquals("SHA1PRNG", ssl.getSecureRandomAlgorithm()); - Assert.assertFalse(ssl.getNeedClientAuth()); - Assert.assertFalse(ssl.getWantClientAuth()); Assert.assertEquals("JKS", ssl.getKeyStoreType()); Assert.assertEquals("JKS", ssl.getTrustStoreType()); Assert.assertEquals("TLS", ssl.getProtocol()); @@ -131,7 +128,7 @@ public void testCreateServerSideSslContextFactoryDefaultValues() { configMap.put("ssl.secure.random.implementation", "SHA1PRNG"); RestServerConfig config = RestServerConfig.forPublic(null, configMap); - SslContextFactory ssl = SSLUtils.createServerSideSslContextFactory(config); + SslContextFactory.Server ssl = SSLUtils.createServerSideSslContextFactory(config); Assert.assertEquals(SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, ssl.getKeyStoreType()); Assert.assertEquals(SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, ssl.getTrustStoreType()); @@ -156,7 +153,7 @@ public void testCreateClientSideSslContextFactoryDefaultValues() { configMap.put("ssl.secure.random.implementation", "SHA1PRNG"); RestServerConfig config = RestServerConfig.forPublic(null, configMap); - SslContextFactory ssl = SSLUtils.createClientSideSslContextFactory(config); + SslContextFactory.Client ssl = SSLUtils.createClientSideSslContextFactory(config); Assert.assertEquals(SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, ssl.getKeyStoreType()); Assert.assertEquals(SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, ssl.getTrustStoreType()); @@ -164,7 +161,5 @@ public void testCreateClientSideSslContextFactoryDefaultValues() { Assert.assertArrayEquals(Arrays.asList(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split("\\s*,\\s*")).toArray(), ssl.getIncludeProtocols()); Assert.assertEquals(SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, ssl.getKeyManagerFactoryAlgorithm()); Assert.assertEquals(SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, ssl.getTrustManagerFactoryAlgorithm()); - Assert.assertFalse(ssl.getNeedClientAuth()); - Assert.assertFalse(ssl.getWantClientAuth()); } } 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 993f5368ff..41091bbdf4 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 @@ -90,6 +90,7 @@ import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; @@ -297,7 +298,7 @@ public void testDestroyConnector() throws Exception { ExecutionException.class, () -> failedDeleteCallback.get(1000L, TimeUnit.MILLISECONDS) ); - assertTrue(e.getCause() instanceof NotFoundException); + assertInstanceOf(NotFoundException.class, e.getCause()); } @Test @@ -482,7 +483,7 @@ public void testRestartConnectorAndTasksUnknownConnector() { RestartRequest restartRequest = new RestartRequest("UnknownConnector", false, true); herder.restartConnectorAndTasks(restartRequest, restartCallback); ExecutionException ee = assertThrows(ExecutionException.class, () -> restartCallback.get(1000L, TimeUnit.MILLISECONDS)); - assertTrue(ee.getCause() instanceof NotFoundException); + assertInstanceOf(NotFoundException.class, ee.getCause()); } @Test @@ -504,7 +505,7 @@ public void testRestartConnectorAndTasksNoStatus() throws Exception { FutureCallback restartCallback = new FutureCallback<>(); herder.restartConnectorAndTasks(restartRequest, restartCallback); ExecutionException ee = assertThrows(ExecutionException.class, () -> restartCallback.get(1000L, TimeUnit.MILLISECONDS)); - assertTrue(ee.getCause() instanceof NotFoundException); + assertInstanceOf(NotFoundException.class, ee.getCause()); assertTrue(ee.getMessage().contains("Status for connector")); } @@ -856,7 +857,7 @@ public void testCorruptConfig() { ); assertNotNull(e.getCause()); Throwable cause = e.getCause(); - assertTrue(cause instanceof BadRequestException); + assertInstanceOf(BadRequestException.class, cause); assertEquals( cause.getMessage(), "Connector configuration is invalid and contains the following 1 error(s):\n" + @@ -910,12 +911,12 @@ public void testModifyConnectorOffsetsUnknownConnector() { Collections.singletonMap(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")), alterOffsetsCallback); ExecutionException e = assertThrows(ExecutionException.class, () -> alterOffsetsCallback.get(1000L, TimeUnit.MILLISECONDS)); - assertTrue(e.getCause() instanceof NotFoundException); + assertInstanceOf(NotFoundException.class, e.getCause()); FutureCallback resetOffsetsCallback = new FutureCallback<>(); herder.resetConnectorOffsets("unknown-connector", resetOffsetsCallback); e = assertThrows(ExecutionException.class, () -> resetOffsetsCallback.get(1000L, TimeUnit.MILLISECONDS)); - assertTrue(e.getCause() instanceof NotFoundException); + assertInstanceOf(NotFoundException.class, e.getCause()); } @Test @@ -938,12 +939,12 @@ public void testModifyConnectorOffsetsConnectorNotInStoppedState() { Collections.singletonMap(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")), alterOffsetsCallback); ExecutionException e = assertThrows(ExecutionException.class, () -> alterOffsetsCallback.get(1000L, TimeUnit.MILLISECONDS)); - assertTrue(e.getCause() instanceof BadRequestException); + assertInstanceOf(BadRequestException.class, e.getCause()); FutureCallback resetOffsetsCallback = new FutureCallback<>(); herder.resetConnectorOffsets(CONNECTOR_NAME, resetOffsetsCallback); e = assertThrows(ExecutionException.class, () -> resetOffsetsCallback.get(1000L, TimeUnit.MILLISECONDS)); - assertTrue(e.getCause() instanceof BadRequestException); + assertInstanceOf(BadRequestException.class, e.getCause()); } @Test 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 78f995253b..a0e4d569f4 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 @@ -65,6 +65,7 @@ import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.mockito.AdditionalMatchers.aryEq; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.isNull; @@ -404,7 +405,7 @@ public void testSetFailure() { callback0.getValue().onCompletion(null, null); ExecutionException e = assertThrows(ExecutionException.class, () -> setFuture.get(10000, TimeUnit.MILLISECONDS)); assertNotNull(e.getCause()); - assertTrue(e.getCause() instanceof KafkaException); + assertInstanceOf(KafkaException.class, e.getCause()); store.stop(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetUtilsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetUtilsTest.java index 06fb51d3ca..ba81a700b4 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetUtilsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetUtilsTest.java @@ -134,6 +134,16 @@ public void testProcessPartitionKeyValidList() { } } + @Test + public void testProcessPartitionKeyNullPartition() { + try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(OffsetUtils.class)) { + Map>> connectorPartitions = new HashMap<>(); + OffsetUtils.processPartitionKey(serializePartitionKey(Arrays.asList("connector-name", null)), new byte[0], CONVERTER, connectorPartitions); + assertEquals(Collections.emptyMap(), connectorPartitions); + assertEquals(0, logCaptureAppender.getMessages().size()); + } + } + private byte[] serializePartitionKey(Object key) { return CONVERTER.fromConnectData("", null, key); } 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 9f78977967..07ccbde9de 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 @@ -76,6 +76,7 @@ import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; public class TopicAdminTest { @@ -110,7 +111,7 @@ public void throwsWithApiVersionMismatchOnDescribe() { env.kafkaClient().prepareResponse(describeTopicResponseWithUnsupportedVersion(newTopic)); TopicAdmin admin = new TopicAdmin(env.adminClient()); Exception e = assertThrows(ConnectException.class, () -> admin.describeTopics(newTopic.name())); - assertTrue(e.getCause() instanceof UnsupportedVersionException); + assertInstanceOf(UnsupportedVersionException.class, e.getCause()); } } @@ -136,7 +137,7 @@ public void throwsWithClusterAuthorizationFailureOnDescribe() { env.kafkaClient().prepareResponse(describeTopicResponseWithClusterAuthorizationException(newTopic)); TopicAdmin admin = new TopicAdmin(env.adminClient()); Exception e = assertThrows(ConnectException.class, () -> admin.describeTopics(newTopic.name())); - assertTrue(e.getCause() instanceof ClusterAuthorizationException); + assertInstanceOf(ClusterAuthorizationException.class, e.getCause()); } } @@ -162,7 +163,7 @@ public void throwsWithTopicAuthorizationFailureOnDescribe() { env.kafkaClient().prepareResponse(describeTopicResponseWithTopicAuthorizationException(newTopic)); TopicAdmin admin = new TopicAdmin(env.adminClient()); Exception e = assertThrows(ConnectException.class, () -> admin.describeTopics(newTopic.name())); - assertTrue(e.getCause() instanceof TopicAuthorizationException); + assertInstanceOf(TopicAuthorizationException.class, e.getCause()); } } 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 index e3694cae29..c15aa27ae5 100644 --- 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 @@ -55,6 +55,7 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.metadata.BrokerState; +import org.apache.kafka.storage.internals.log.CleanerConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -160,7 +161,7 @@ private void doStart() { putIfAbsent(brokerConfig, KafkaConfig.OffsetsTopicReplicationFactorProp(), (short) brokers.length); putIfAbsent(brokerConfig, KafkaConfig.AutoCreateTopicsEnableProp(), false); // reduce the size of the log cleaner map to reduce test memory usage - putIfAbsent(brokerConfig, KafkaConfig.LogCleanerDedupeBufferSizeProp(), 2 * 1024 * 1024L); + putIfAbsent(brokerConfig, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, 2 * 1024 * 1024L); Object listenerConfig = brokerConfig.get(KafkaConfig.InterBrokerListenerNameProp()); if (listenerConfig == null) 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 b5729c1929..1017518efa 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 @@ -34,10 +34,10 @@ import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; public class FlattenTest { private final Flatten xformKey = new Flatten.Key<>(); @@ -145,7 +145,7 @@ public void testNestedMapWithDelimiter() { null, twoLevelNestedMap)); assertNull(transformed.valueSchema()); - assertTrue(transformed.value() instanceof Map); + assertInstanceOf(Map.class, transformed.value()); @SuppressWarnings("unchecked") Map transformedMap = (Map) transformed.value(); assertEquals(9, transformedMap.size()); @@ -241,7 +241,7 @@ public void testOptionalFieldMap() { null, oneLevelNestedMap)); assertNull(transformed.valueSchema()); - assertTrue(transformed.value() instanceof Map); + assertInstanceOf(Map.class, transformed.value()); @SuppressWarnings("unchecked") Map transformedMap = (Map) transformed.value(); @@ -257,7 +257,7 @@ public void testKey() { SourceRecord transformed = xformKey.apply(src); assertNull(transformed.keySchema()); - assertTrue(transformed.key() instanceof Map); + assertInstanceOf(Map.class, transformed.key()); @SuppressWarnings("unchecked") Map transformedMap = (Map) transformed.key(); assertEquals(12, transformedMap.get("A.B")); diff --git a/core/src/main/scala/kafka/MetadataLogConfig.scala b/core/src/main/scala/kafka/MetadataLogConfig.scala new file mode 100755 index 0000000000..60afac20fa --- /dev/null +++ b/core/src/main/scala/kafka/MetadataLogConfig.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.raft + +import kafka.server.KafkaConfig +import org.apache.kafka.common.config.AbstractConfig +import org.apache.kafka.storage.internals.log.LogConfig + +final case class MetadataLogConfig( + logSegmentBytes: Int, + logSegmentMinBytes: Int, + logSegmentMillis: Long, + retentionMaxBytes: Long, + retentionMillis: Long, + maxBatchSizeInBytes: Int, + maxFetchSizeInBytes: Int, + fileDeleteDelayMs: Long, + nodeId: Int +) + +object MetadataLogConfig { + def apply(config: AbstractConfig, maxBatchSizeInBytes: Int, maxFetchSizeInBytes: Int): MetadataLogConfig = { + new MetadataLogConfig( + config.getInt(KafkaConfig.MetadataLogSegmentBytesProp), + config.getInt(KafkaConfig.MetadataLogSegmentMinBytesProp), + config.getLong(KafkaConfig.MetadataLogSegmentMillisProp), + config.getLong(KafkaConfig.MetadataMaxRetentionBytesProp), + config.getLong(KafkaConfig.MetadataMaxRetentionMillisProp), + maxBatchSizeInBytes, + maxFetchSizeInBytes, + LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, + config.getInt(KafkaConfig.NodeIdProp) + ) + } +} diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala deleted file mode 100755 index 4187274a22..0000000000 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ /dev/null @@ -1,1167 +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 com.fasterxml.jackson.databind.{ObjectReader, ObjectWriter} - -import java.time.{Duration, Instant} -import java.util.{Collections, Properties} -import com.fasterxml.jackson.dataformat.csv.CsvMapper -import com.fasterxml.jackson.module.scala.DefaultScalaModule -import kafka.utils._ -import kafka.utils.Implicits._ -import org.apache.kafka.clients.admin._ -import org.apache.kafka.clients.consumer.OffsetAndMetadata -import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.common.utils.Utils -import org.apache.kafka.common.{KafkaException, Node, TopicPartition} -import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils} - -import scala.jdk.CollectionConverters._ -import scala.collection.mutable.ListBuffer -import scala.collection.{Map, Seq, immutable, mutable} -import scala.util.{Failure, Success, Try} -import joptsimple.{OptionException, OptionSpec, OptionSpecBuilder} -import org.apache.kafka.common.protocol.Errors - -import scala.collection.immutable.TreeMap -import scala.reflect.ClassTag -import org.apache.kafka.common.ConsumerGroupState -import org.apache.kafka.common.requests.ListOffsetsResponse - -object ConsumerGroupCommand extends Logging { - - def main(args: Array[String]): Unit = { - - val opts = new ConsumerGroupCommandOptions(args) - try { - opts.checkArgs() - CommandLineUtils.maybePrintHelpOrVersion(opts, "This tool helps to list all consumer groups, describe a consumer group, delete consumer group info, or reset consumer group offsets.") - - // should have exactly one action - val actions = Seq(opts.listOpt, opts.describeOpt, opts.deleteOpt, opts.resetOffsetsOpt, opts.deleteOffsetsOpt).count(opts.options.has) - if (actions != 1) - CommandLineUtils.printUsageAndExit(opts.parser, "Command must include exactly one action: --list, --describe, --delete, --reset-offsets, --delete-offsets") - - run(opts) - } catch { - case e: OptionException => - CommandLineUtils.printUsageAndExit(opts.parser, e.getMessage) - } - } - - def run(opts: ConsumerGroupCommandOptions): Unit = { - val consumerGroupService = new ConsumerGroupService(opts) - try { - if (opts.options.has(opts.listOpt)) - consumerGroupService.listGroups() - else if (opts.options.has(opts.describeOpt)) - consumerGroupService.describeGroups() - else if (opts.options.has(opts.deleteOpt)) - consumerGroupService.deleteGroups() - else if (opts.options.has(opts.resetOffsetsOpt)) { - val offsetsToReset = consumerGroupService.resetOffsets() - if (opts.options.has(opts.exportOpt)) { - val exported = consumerGroupService.exportOffsetsToCsv(offsetsToReset) - println(exported) - } else - printOffsetsToReset(offsetsToReset) - } - else if (opts.options.has(opts.deleteOffsetsOpt)) { - consumerGroupService.deleteOffsets() - } - } catch { - case e: IllegalArgumentException => - CommandLineUtils.printUsageAndExit(opts.parser, e.getMessage) - case e: Throwable => - printError(s"Executing consumer group command failed due to ${e.getMessage}", Some(e)) - } finally { - consumerGroupService.close() - } - } - - def consumerGroupStatesFromString(input: String): Set[ConsumerGroupState] = { - val parsedStates = input.split(',').map(s => ConsumerGroupState.parse(s.trim)).toSet - if (parsedStates.contains(ConsumerGroupState.UNKNOWN)) { - val validStates = ConsumerGroupState.values().filter(_ != ConsumerGroupState.UNKNOWN) - throw new IllegalArgumentException(s"Invalid state list '$input'. Valid states are: ${validStates.mkString(", ")}") - } - parsedStates - } - - val MISSING_COLUMN_VALUE = "-" - - private def printError(msg: String, e: Option[Throwable] = None): Unit = { - println(s"\nError: $msg") - e.foreach(_.printStackTrace()) - } - - private def printOffsetsToReset(groupAssignmentsToReset: Map[String, Map[TopicPartition, OffsetAndMetadata]]): Unit = { - val format = "%-30s %-30s %-10s %-15s" - if (groupAssignmentsToReset.nonEmpty) - println("\n" + format.format("GROUP", "TOPIC", "PARTITION", "NEW-OFFSET")) - for { - (groupId, assignment) <- groupAssignmentsToReset - (consumerAssignment, offsetAndMetadata) <- assignment - } { - println(format.format( - groupId, - consumerAssignment.topic, - consumerAssignment.partition, - offsetAndMetadata.offset)) - } - } - - 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]) - - private[admin] case class MemberAssignmentState(group: String, consumerId: String, host: String, clientId: String, groupInstanceId: String, - numPartitions: Int, assignment: List[TopicPartition]) - - case class GroupState(group: String, coordinator: Node, assignmentStrategy: String, state: String, numMembers: Int) - - private[admin] sealed trait CsvRecord - private[admin] case class CsvRecordWithGroup(group: String, topic: String, partition: Int, offset: Long) extends CsvRecord - private[admin] case class CsvRecordNoGroup(topic: String, partition: Int, offset: Long) extends CsvRecord - private[admin] object CsvRecordWithGroup { - val fields: Array[String] = Array("group", "topic", "partition", "offset") - } - private[admin] object CsvRecordNoGroup { - val fields: Array[String] = Array("topic", "partition", "offset") - } - // Example: CsvUtils().readerFor[CsvRecordWithoutGroup] - private[admin] case class CsvUtils() { - val mapper = new CsvMapper - mapper.registerModule(DefaultScalaModule) - def readerFor[T <: CsvRecord : ClassTag]: ObjectReader = { - val schema = getSchema[T] - val clazz = implicitly[ClassTag[T]].runtimeClass - mapper.readerFor(clazz).`with`(schema) - } - def writerFor[T <: CsvRecord : ClassTag]: ObjectWriter = { - val schema = getSchema[T] - val clazz = implicitly[ClassTag[T]].runtimeClass - mapper.writerFor(clazz).`with`(schema) - } - private def getSchema[T <: CsvRecord : ClassTag] = { - val clazz = implicitly[ClassTag[T]].runtimeClass - - val fields = - if (classOf[CsvRecordWithGroup] == clazz) CsvRecordWithGroup.fields - else if (classOf[CsvRecordNoGroup] == clazz) CsvRecordNoGroup.fields - else throw new IllegalStateException(s"Unhandled class $clazz") - - val schema = mapper.schemaFor(clazz).sortedBy(fields: _*) - schema - } - } - - class ConsumerGroupService(val opts: ConsumerGroupCommandOptions, - private[admin] val configOverrides: Map[String, String] = Map.empty) { - - private val adminClient = createAdminClient(configOverrides) - - // We have to make sure it is evaluated once and available - private lazy val resetPlanFromFile: Option[Map[String, Map[TopicPartition, OffsetAndMetadata]]] = { - if (opts.options.has(opts.resetFromFileOpt)) { - val resetPlanPath = opts.options.valueOf(opts.resetFromFileOpt) - val resetPlanCsv = Utils.readFileAsString(resetPlanPath) - val resetPlan = parseResetPlan(resetPlanCsv) - Some(resetPlan) - } else None - } - - def listGroups(): Unit = { - if (opts.options.has(opts.stateOpt)) { - val stateValue = opts.options.valueOf(opts.stateOpt) - val states = if (stateValue == null || stateValue.isEmpty) - Set[ConsumerGroupState]() - else - consumerGroupStatesFromString(stateValue) - val listings = listConsumerGroupsWithState(states) - printGroupStates(listings.map(e => (e.groupId, e.state.get.toString))) - } else - listConsumerGroups().foreach(println(_)) - } - - def listConsumerGroups(): List[String] = { - val result = adminClient.listConsumerGroups(withTimeoutMs(new ListConsumerGroupsOptions)) - val listings = result.all.get.asScala - listings.map(_.groupId).toList - } - - def listConsumerGroupsWithState(states: Set[ConsumerGroupState]): List[ConsumerGroupListing] = { - val listConsumerGroupsOptions = withTimeoutMs(new ListConsumerGroupsOptions()) - listConsumerGroupsOptions.inStates(states.asJava) - val result = adminClient.listConsumerGroups(listConsumerGroupsOptions) - result.all.get.asScala.toList - } - - private def printGroupStates(groupsAndStates: List[(String, String)]): Unit = { - // find proper columns width - var maxGroupLen = 15 - for ((groupId, _) <- groupsAndStates) { - maxGroupLen = Math.max(maxGroupLen, groupId.length) - } - val format = s"%${-maxGroupLen}s %s" - println(format.format("GROUP", "STATE")) - for ((groupId, state) <- groupsAndStates) { - println(format.format(groupId, state)) - } - } - - 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 => - printError(s"The consumer group '$group' does not exist.") - false - case Some(num) => state match { - case Some("Dead") => - printError(s"Consumer group '$group' does not exist.") - case Some("Empty") => - Console.err.println(s"\nConsumer group '$group' has no active members.") - case Some("PreparingRebalance") | Some("CompletingRebalance") | Some("Assigning") | Some("Reconciling") => - Console.err.println(s"\nWarning: 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 - } - } - - private def size(colOpt: Option[Seq[Object]]): Option[Int] = colOpt.map(_.size) - - private def printOffsets(offsets: Map[String, (Option[String], Option[Seq[PartitionAssignmentState]])]): Unit = { - for ((groupId, (state, assignments)) <- offsets) { - if (shouldPrintMemberState(groupId, state, size(assignments))) { - // find proper columns width - var (maxGroupLen, maxTopicLen, maxConsumerIdLen, maxHostLen) = (15, 15, 15, 15) - assignments match { - case None => // do nothing - case Some(consumerAssignments) => - consumerAssignments.foreach { consumerAssignment => - maxGroupLen = Math.max(maxGroupLen, consumerAssignment.group.length) - maxTopicLen = Math.max(maxTopicLen, consumerAssignment.topic.getOrElse(MISSING_COLUMN_VALUE).length) - maxConsumerIdLen = Math.max(maxConsumerIdLen, consumerAssignment.consumerId.getOrElse(MISSING_COLUMN_VALUE).length) - maxHostLen = Math.max(maxHostLen, consumerAssignment.host.getOrElse(MISSING_COLUMN_VALUE).length) - } - } - - val format = s"%${-maxGroupLen}s %${-maxTopicLen}s %-10s %-15s %-15s %-15s %${-maxConsumerIdLen}s %${-maxHostLen}s %s" - println("\n" + format - .format("GROUP", "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 => - println(format.format( - consumerAssignment.group, - 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), - consumerAssignment.host.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.clientId.getOrElse(MISSING_COLUMN_VALUE)) - ) - } - } - } - } - } - - private def printMembers(members: Map[String, (Option[String], Option[Seq[MemberAssignmentState]])], verbose: Boolean): Unit = { - for ((groupId, (state, assignments)) <- members) { - if (shouldPrintMemberState(groupId, state, size(assignments))) { - // find proper columns width - var (maxGroupLen, maxConsumerIdLen, maxGroupInstanceIdLen, maxHostLen, maxClientIdLen, includeGroupInstanceId) = (15, 15, 17, 15, 15, false) - assignments match { - case None => // do nothing - case Some(memberAssignments) => - memberAssignments.foreach { memberAssignment => - maxGroupLen = Math.max(maxGroupLen, memberAssignment.group.length) - maxConsumerIdLen = Math.max(maxConsumerIdLen, memberAssignment.consumerId.length) - maxGroupInstanceIdLen = Math.max(maxGroupInstanceIdLen, memberAssignment.groupInstanceId.length) - maxHostLen = Math.max(maxHostLen, memberAssignment.host.length) - maxClientIdLen = Math.max(maxClientIdLen, memberAssignment.clientId.length) - includeGroupInstanceId = includeGroupInstanceId || memberAssignment.groupInstanceId.nonEmpty - } - } - - val wideFormat = s"%${-maxGroupLen}s %${-maxConsumerIdLen}s %${-maxGroupInstanceIdLen}s %${-maxHostLen}s %${-maxClientIdLen}s %-15s " - val shortFormat = s"%${-maxGroupLen}s %${-maxConsumerIdLen}s %${-maxHostLen}s %${-maxClientIdLen}s %-15s " - - if (includeGroupInstanceId) { - print("\n" + wideFormat - .format("GROUP", "CONSUMER-ID", "GROUP-INSTANCE-ID", "HOST", "CLIENT-ID", "#PARTITIONS")) - } else { - print("\n" + shortFormat - .format("GROUP", "CONSUMER-ID", "HOST", "CLIENT-ID", "#PARTITIONS")) - } - if (verbose) - print(s"%s".format("ASSIGNMENT")) - println() - - assignments match { - case None => // do nothing - case Some(memberAssignments) => - memberAssignments.foreach { memberAssignment => - if (includeGroupInstanceId) { - print(wideFormat.format( - memberAssignment.group, memberAssignment.consumerId, memberAssignment.groupInstanceId, memberAssignment.host, - memberAssignment.clientId, memberAssignment.numPartitions)) - } else { - print(shortFormat.format( - memberAssignment.group, memberAssignment.consumerId, memberAssignment.host, memberAssignment.clientId, memberAssignment.numPartitions)) - } - if (verbose) { - val partitions = memberAssignment.assignment match { - case List() => MISSING_COLUMN_VALUE - case assignment => - assignment.groupBy(_.topic).map { - case (topic, partitionList) => topic + partitionList.map(_.partition).sorted.mkString("(", ",", ")") - }.toList.sorted.mkString(", ") - } - print(s"%s".format(partitions)) - } - println() - } - } - } - } - } - - private def printStates(states: Map[String, GroupState]): Unit = { - for ((groupId, state) <- states) { - if (shouldPrintMemberState(groupId, Some(state.state), Some(1))) { - val coordinator = s"${state.coordinator.host}:${state.coordinator.port} (${state.coordinator.idString})" - val coordinatorColLen = Math.max(25, coordinator.length) - val format = s"\n%${-coordinatorColLen}s %-25s %-20s %-15s %s" - print(format.format("GROUP", "COORDINATOR (ID)", "ASSIGNMENT-STRATEGY", "STATE", "#MEMBERS")) - print(format.format(state.group, coordinator, state.assignmentStrategy, state.state, state.numMembers)) - println() - } - } - } - - def describeGroups(): Unit = { - val groupIds = - if (opts.options.has(opts.allGroupsOpt)) listConsumerGroups() - else opts.options.valuesOf(opts.groupOpt).asScala - val membersOptPresent = opts.options.has(opts.membersOpt) - val stateOptPresent = opts.options.has(opts.stateOpt) - val offsetsOptPresent = opts.options.has(opts.offsetsOpt) - val subActions = Seq(membersOptPresent, offsetsOptPresent, stateOptPresent).count(_ == true) - - if (subActions == 0 || offsetsOptPresent) { - val offsets = collectGroupsOffsets(groupIds) - printOffsets(offsets) - } else if (membersOptPresent) { - val members = collectGroupsMembers(groupIds, opts.options.has(opts.verboseOpt)) - printMembers(members, opts.options.has(opts.verboseOpt)) - } else { - val states = collectGroupsState(groupIds) - printStates(states) - } - } - - private def collectConsumerAssignment(group: String, - coordinator: Option[Node], - topicPartitions: Seq[TopicPartition], - getPartitionOffset: TopicPartition => Option[Long], - consumerIdOpt: Option[String], - hostOpt: Option[String], - clientIdOpt: Option[String]): Array[PartitionAssignmentState] = { - if (topicPartitions.isEmpty) { - Array[PartitionAssignmentState]( - PartitionAssignmentState(group, coordinator, None, None, None, getLag(None, None), consumerIdOpt, hostOpt, clientIdOpt, None) - ) - } - else - describePartitions(group, coordinator, topicPartitions.sortBy(_.partition), getPartitionOffset, consumerIdOpt, hostOpt, clientIdOpt) - } - - private def getLag(offset: Option[Long], logEndOffset: Option[Long]): Option[Long] = - offset.filter(_ != -1).flatMap(offset => logEndOffset.map(_ - offset)) - - private def describePartitions(group: String, - coordinator: Option[Node], - topicPartitions: Seq[TopicPartition], - getPartitionOffset: TopicPartition => Option[Long], - consumerIdOpt: Option[String], - hostOpt: Option[String], - clientIdOpt: Option[String]): Array[PartitionAssignmentState] = { - - def getDescribePartitionResult(topicPartition: TopicPartition, logEndOffsetOpt: Option[Long]): PartitionAssignmentState = { - val offset = getPartitionOffset(topicPartition) - PartitionAssignmentState(group, coordinator, Option(topicPartition.topic), Option(topicPartition.partition), offset, - getLag(offset, logEndOffsetOpt), consumerIdOpt, hostOpt, clientIdOpt, logEndOffsetOpt) - } - - 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 - } - }.toArray - } - - def resetOffsets(): Map[String, Map[TopicPartition, OffsetAndMetadata]] = { - val groupIds = - if (opts.options.has(opts.allGroupsOpt)) listConsumerGroups() - else opts.options.valuesOf(opts.groupOpt).asScala - - val consumerGroups = adminClient.describeConsumerGroups( - groupIds.asJava, - withTimeoutMs(new DescribeConsumerGroupsOptions) - ).describedGroups() - - val result = - consumerGroups.asScala.foldLeft(immutable.Map[String, Map[TopicPartition, OffsetAndMetadata]]()) { - case (acc, (groupId, groupDescription)) => - groupDescription.get.state().toString match { - case "Empty" | "Dead" => - val partitionsToReset = getPartitionsToReset(groupId) - val preparedOffsets = prepareOffsetsToReset(groupId, partitionsToReset) - - // Dry-run is the default behavior if --execute is not specified - val dryRun = opts.options.has(opts.dryRunOpt) || !opts.options.has(opts.executeOpt) - if (!dryRun) { - adminClient.alterConsumerGroupOffsets( - groupId, - preparedOffsets.asJava, - withTimeoutMs(new AlterConsumerGroupOffsetsOptions) - ).all.get - } - acc.updated(groupId, preparedOffsets) - case currentState => - printError(s"Assignments can only be reset if the group '$groupId' is inactive, but the current state is $currentState.") - acc.updated(groupId, Map.empty) - } - } - result - } - - def deleteOffsets(groupId: String, topics: List[String]): (Errors, Map[TopicPartition, Throwable]) = { - val partitionLevelResult = mutable.Map[TopicPartition, Throwable]() - - val (topicWithPartitions, topicWithoutPartitions) = topics.partition(_.contains(":")) - val knownPartitions = topicWithPartitions.flatMap(parseTopicsWithPartitions) - - // Get the partitions of topics that the user did not explicitly specify the partitions - val describeTopicsResult = adminClient.describeTopics( - topicWithoutPartitions.asJava, - withTimeoutMs(new DescribeTopicsOptions)) - - val unknownPartitions = describeTopicsResult.topicNameValues().asScala.flatMap { case (topic, future) => - Try(future.get()) match { - case Success(description) => description.partitions().asScala.map { partition => - new TopicPartition(topic, partition.partition()) - } - case Failure(e) => - partitionLevelResult += new TopicPartition(topic, -1) -> e - List.empty - } - } - - val partitions = knownPartitions ++ unknownPartitions - - val deleteResult = adminClient.deleteConsumerGroupOffsets( - groupId, - partitions.toSet.asJava, - withTimeoutMs(new DeleteConsumerGroupOffsetsOptions) - ) - - var topLevelException = Errors.NONE - Try(deleteResult.all.get) match { - case Success(_) => - case Failure(e) => topLevelException = Errors.forException(e.getCause) - } - - partitions.foreach { partition => - Try(deleteResult.partitionResult(partition).get()) match { - case Success(_) => partitionLevelResult += partition -> null - case Failure(e) => partitionLevelResult += partition -> e - } - } - - (topLevelException, partitionLevelResult) - } - - def deleteOffsets(): Unit = { - val groupId = opts.options.valueOf(opts.groupOpt) - val topics = opts.options.valuesOf(opts.topicOpt).asScala.toList - - val (topLevelResult, partitionLevelResult) = deleteOffsets(groupId, topics) - - topLevelResult match { - case Errors.NONE => - println(s"Request succeed for deleting offsets with topic ${topics.mkString(", ")} group $groupId") - case Errors.INVALID_GROUP_ID => - printError(s"'$groupId' is not valid.") - case Errors.GROUP_ID_NOT_FOUND => - printError(s"'$groupId' does not exist.") - case Errors.GROUP_AUTHORIZATION_FAILED => - printError(s"Access to '$groupId' is not authorized.") - case Errors.NON_EMPTY_GROUP => - printError(s"Deleting offsets of a consumer group '$groupId' is forbidden if the group is not empty.") - case Errors.GROUP_SUBSCRIBED_TO_TOPIC | - Errors.TOPIC_AUTHORIZATION_FAILED | - Errors.UNKNOWN_TOPIC_OR_PARTITION => - printError(s"Encounter some partition level error, see the follow-up details:") - case _ => - printError(s"Encounter some unknown error: $topLevelResult") - } - - val format = "%-30s %-15s %-15s" - println("\n" + format.format("TOPIC", "PARTITION", "STATUS")) - partitionLevelResult.toList.sortBy(t => t._1.topic + t._1.partition.toString).foreach { case (tp, error) => - println(format.format( - tp.topic, - if (tp.partition >= 0) tp.partition else "Not Provided", - if (error != null) s"Error: ${error.getMessage}" else "Successful" - )) - } - } - - private[admin] def describeConsumerGroups(groupIds: Seq[String]): mutable.Map[String, ConsumerGroupDescription] = { - adminClient.describeConsumerGroups( - groupIds.asJava, - withTimeoutMs(new DescribeConsumerGroupsOptions) - ).describedGroups().asScala.map { - case (groupId, groupDescriptionFuture) => (groupId, groupDescriptionFuture.get()) - } - } - - /** - * Returns the state of the specified consumer group and partition assignment states - */ - def collectGroupOffsets(groupId: String): (Option[String], Option[Seq[PartitionAssignmentState]]) = { - collectGroupsOffsets(List(groupId)).getOrElse(groupId, (None, None)) - } - - /** - * Returns states of the specified consumer groups and partition assignment states - */ - private def collectGroupsOffsets(groupIds: Seq[String]): TreeMap[String, (Option[String], Option[Seq[PartitionAssignmentState]])] = { - val consumerGroups = describeConsumerGroups(groupIds) - - val groupOffsets = TreeMap[String, (Option[String], Option[Seq[PartitionAssignmentState]])]() ++ (for ((groupId, consumerGroup) <- consumerGroups) yield { - val state = consumerGroup.state - val committedOffsets = getCommittedOffsets(groupId) - // The admin client returns `null` as a value to indicate that there is not committed offset for a partition. - def getPartitionOffset(tp: TopicPartition): Option[Long] = committedOffsets.get(tp).filter(_ != null).map(_.offset) - var assignedTopicPartitions = ListBuffer[TopicPartition]() - val rowsWithConsumer = consumerGroup.members.asScala.filterNot(_.assignment.topicPartitions.isEmpty).toSeq - .sortBy(_.assignment.topicPartitions.size)(Ordering[Int].reverse).flatMap { consumerSummary => - val topicPartitions = consumerSummary.assignment.topicPartitions.asScala - assignedTopicPartitions = assignedTopicPartitions ++ topicPartitions - collectConsumerAssignment(groupId, Option(consumerGroup.coordinator), topicPartitions.toList, - getPartitionOffset, Some(s"${consumerSummary.consumerId}"), Some(s"${consumerSummary.host}"), - Some(s"${consumerSummary.clientId}")) - } - val unassignedPartitions = committedOffsets.filterNot { case (tp, _) => assignedTopicPartitions.contains(tp) } - val rowsWithoutConsumer = if (unassignedPartitions.nonEmpty) { - collectConsumerAssignment( - groupId, - Option(consumerGroup.coordinator), - unassignedPartitions.keySet.toSeq, - getPartitionOffset, - Some(MISSING_COLUMN_VALUE), - Some(MISSING_COLUMN_VALUE), - Some(MISSING_COLUMN_VALUE)).toSeq - } else - Seq.empty - - groupId -> (Some(state.toString), Some(rowsWithConsumer ++ rowsWithoutConsumer)) - }).toMap - - groupOffsets - } - - private[admin] def collectGroupMembers(groupId: String, verbose: Boolean): (Option[String], Option[Seq[MemberAssignmentState]]) = { - collectGroupsMembers(Seq(groupId), verbose)(groupId) - } - - private[admin] def collectGroupsMembers(groupIds: Seq[String], verbose: Boolean): TreeMap[String, (Option[String], Option[Seq[MemberAssignmentState]])] = { - val consumerGroups = describeConsumerGroups(groupIds) - TreeMap[String, (Option[String], Option[Seq[MemberAssignmentState]])]() ++ (for ((groupId, consumerGroup) <- consumerGroups) yield { - val state = consumerGroup.state.toString - val memberAssignmentStates = consumerGroup.members().asScala.map(consumer => - MemberAssignmentState( - groupId, - consumer.consumerId, - consumer.host, - consumer.clientId, - consumer.groupInstanceId.orElse(""), - consumer.assignment.topicPartitions.size(), - if (verbose) consumer.assignment.topicPartitions.asScala.toList else List() - )).toList - groupId -> (Some(state), Option(memberAssignmentStates)) - }).toMap - } - - private[admin] def collectGroupState(groupId: String): GroupState = { - collectGroupsState(Seq(groupId))(groupId) - } - - private[admin] def collectGroupsState(groupIds: Seq[String]): TreeMap[String, GroupState] = { - val consumerGroups = describeConsumerGroups(groupIds) - TreeMap[String, GroupState]() ++ (for ((groupId, groupDescription) <- consumerGroups) yield { - groupId -> GroupState( - groupId, - groupDescription.coordinator, - groupDescription.partitionAssignor(), - groupDescription.state.toString, - groupDescription.members().size - ) - }).toMap - } - - private def getLogEndOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = { - val endOffsets = topicPartitions.map { topicPartition => - topicPartition -> OffsetSpec.latest - }.toMap - val offsets = adminClient.listOffsets( - endOffsets.asJava, - withTimeoutMs(new ListOffsetsOptions) - ).all.get - topicPartitions.map { topicPartition => - Option(offsets.get(topicPartition)) match { - case Some(listOffsetsResultInfo) => topicPartition -> LogOffsetResult.LogOffset(listOffsetsResultInfo.offset) - case _ => topicPartition -> LogOffsetResult.Unknown - } - }.toMap - } - - private def getLogStartOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = { - val startOffsets = topicPartitions.map { topicPartition => - topicPartition -> OffsetSpec.earliest - }.toMap - val offsets = adminClient.listOffsets( - startOffsets.asJava, - withTimeoutMs(new ListOffsetsOptions) - ).all.get - topicPartitions.map { topicPartition => - Option(offsets.get(topicPartition)) match { - case Some(listOffsetsResultInfo) => topicPartition -> LogOffsetResult.LogOffset(listOffsetsResultInfo.offset) - case _ => topicPartition -> LogOffsetResult.Unknown - } - }.toMap - } - - private def getLogTimestampOffsets(topicPartitions: Seq[TopicPartition], timestamp: java.lang.Long): Map[TopicPartition, LogOffsetResult] = { - val timestampOffsets = topicPartitions.map { topicPartition => - topicPartition -> OffsetSpec.forTimestamp(timestamp) - }.toMap - val offsets = adminClient.listOffsets( - timestampOffsets.asJava, - withTimeoutMs(new ListOffsetsOptions) - ).all.get - val (successfulOffsetsForTimes, unsuccessfulOffsetsForTimes) = - offsets.asScala.partition(_._2.offset != ListOffsetsResponse.UNKNOWN_OFFSET) - - val successfulLogTimestampOffsets = successfulOffsetsForTimes.map { - case (topicPartition, listOffsetsResultInfo) => topicPartition -> LogOffsetResult.LogOffset(listOffsetsResultInfo.offset) - }.toMap - - unsuccessfulOffsetsForTimes.foreach { entry => - println(s"\nWarn: Partition " + entry._1.partition() + " from topic " + entry._1.topic() + - " is empty. Falling back to latest known offset.") - } - - successfulLogTimestampOffsets ++ getLogEndOffsets(unsuccessfulOffsetsForTimes.keySet.toSeq) - } - - def close(): Unit = { - adminClient.close() - } - - // Visibility for testing - protected def createAdminClient(configOverrides: Map[String, String]): Admin = { - val props = if (opts.options.has(opts.commandConfigOpt)) Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt)) else new Properties() - props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt)) - configOverrides.forKeyValue { (k, v) => props.put(k, v)} - Admin.create(props) - } - - private def withTimeoutMs [T <: AbstractOptions[T]] (options : T) = { - val t = opts.options.valueOf(opts.timeoutMsOpt).intValue() - options.timeoutMs(t) - } - - private def parseTopicsWithPartitions(topicArg: String): Seq[TopicPartition] = { - def partitionNum(partition: String): Int = { - try { - partition.toInt - } catch { - case _: NumberFormatException => - throw new IllegalArgumentException(s"Invalid partition '$partition' specified in topic arg '$topicArg''") - } - } - topicArg.split(":") match { - case Array(topic, partitions) => - partitions.split(",").map(partition => new TopicPartition(topic, partitionNum(partition))) - case _ => - throw new IllegalArgumentException(s"Invalid topic arg '$topicArg', expected topic name and partitions") - } - } - - private def parseTopicPartitionsToReset(topicArgs: Seq[String]): Seq[TopicPartition] = { - val (topicsWithPartitions, topics) = topicArgs.partition(_.contains(":")) - val specifiedPartitions = topicsWithPartitions.flatMap(parseTopicsWithPartitions) - - val unspecifiedPartitions = if (topics.nonEmpty) { - val descriptionMap = adminClient.describeTopics( - topics.asJava, - withTimeoutMs(new DescribeTopicsOptions) - ).allTopicNames().get.asScala - descriptionMap.flatMap { case (topic, description) => - description.partitions().asScala.map { tpInfo => - new TopicPartition(topic, tpInfo.partition) - } - } - } else - Seq.empty - specifiedPartitions ++ unspecifiedPartitions - } - - private def getPartitionsToReset(groupId: String): Seq[TopicPartition] = { - if (opts.options.has(opts.allTopicsOpt)) { - getCommittedOffsets(groupId).keys.toSeq - } else if (opts.options.has(opts.topicOpt)) { - val topics = opts.options.valuesOf(opts.topicOpt).asScala - parseTopicPartitionsToReset(topics) - } else { - if (opts.options.has(opts.resetFromFileOpt)) - Nil - else - ToolsUtils.printUsageAndExit(opts.parser, "One of the reset scopes should be defined: --all-topics, --topic.") - } - } - - private def getCommittedOffsets(groupId: String): Map[TopicPartition, OffsetAndMetadata] = { - adminClient.listConsumerGroupOffsets( - Collections.singletonMap(groupId, new ListConsumerGroupOffsetsSpec), - withTimeoutMs(new ListConsumerGroupOffsetsOptions()) - ).partitionsToOffsetAndMetadata(groupId).get().asScala - } - - type GroupMetadata = immutable.Map[String, immutable.Map[TopicPartition, OffsetAndMetadata]] - private def parseResetPlan(resetPlanCsv: String): GroupMetadata = { - def updateGroupMetadata(group: String, topic: String, partition: Int, offset: Long, acc: GroupMetadata) = { - val topicPartition = new TopicPartition(topic, partition) - val offsetAndMetadata = new OffsetAndMetadata(offset) - val dataMap = acc.getOrElse(group, immutable.Map()).updated(topicPartition, offsetAndMetadata) - acc.updated(group, dataMap) - } - val csvReader = CsvUtils().readerFor[CsvRecordNoGroup] - val lines = resetPlanCsv.split("\n") - val isSingleGroupQuery = opts.options.valuesOf(opts.groupOpt).size() == 1 - val isOldCsvFormat = lines.headOption.flatMap(line => - Try(csvReader.readValue[CsvRecordNoGroup](line)).toOption).nonEmpty - // Single group CSV format: "topic,partition,offset" - val dataMap = if (isSingleGroupQuery && isOldCsvFormat) { - val group = opts.options.valueOf(opts.groupOpt) - lines.foldLeft(immutable.Map[String, immutable.Map[TopicPartition, OffsetAndMetadata]]()) { (acc, line) => - val CsvRecordNoGroup(topic, partition, offset) = csvReader.readValue[CsvRecordNoGroup](line) - updateGroupMetadata(group, topic, partition, offset, acc) - } - // Multiple group CSV format: "group,topic,partition,offset" - } else { - val csvReader = CsvUtils().readerFor[CsvRecordWithGroup] - lines.foldLeft(immutable.Map[String, immutable.Map[TopicPartition, OffsetAndMetadata]]()) { (acc, line) => - val CsvRecordWithGroup(group, topic, partition, offset) = csvReader.readValue[CsvRecordWithGroup](line) - updateGroupMetadata(group, topic, partition, offset, acc) - } - } - dataMap - } - - private def prepareOffsetsToReset(groupId: String, - partitionsToReset: Seq[TopicPartition]): Map[TopicPartition, OffsetAndMetadata] = { - if (opts.options.has(opts.resetToOffsetOpt)) { - val offset = opts.options.valueOf(opts.resetToOffsetOpt) - checkOffsetsRange(partitionsToReset.map((_, offset)).toMap).map { - case (topicPartition, newOffset) => (topicPartition, new OffsetAndMetadata(newOffset)) - } - } else if (opts.options.has(opts.resetToEarliestOpt)) { - val logStartOffsets = getLogStartOffsets(partitionsToReset) - partitionsToReset.map { topicPartition => - logStartOffsets.get(topicPartition) match { - case Some(LogOffsetResult.LogOffset(offset)) => (topicPartition, new OffsetAndMetadata(offset)) - case _ => ToolsUtils.printUsageAndExit(opts.parser, s"Error getting starting offset of topic partition: $topicPartition") - } - }.toMap - } else if (opts.options.has(opts.resetToLatestOpt)) { - val logEndOffsets = getLogEndOffsets(partitionsToReset) - partitionsToReset.map { topicPartition => - logEndOffsets.get(topicPartition) match { - case Some(LogOffsetResult.LogOffset(offset)) => (topicPartition, new OffsetAndMetadata(offset)) - case _ => ToolsUtils.printUsageAndExit(opts.parser, s"Error getting ending offset of topic partition: $topicPartition") - } - }.toMap - } else if (opts.options.has(opts.resetShiftByOpt)) { - val currentCommittedOffsets = getCommittedOffsets(groupId) - val requestedOffsets = partitionsToReset.map { topicPartition => - val shiftBy = opts.options.valueOf(opts.resetShiftByOpt) - val currentOffset = currentCommittedOffsets.getOrElse(topicPartition, - throw new IllegalArgumentException(s"Cannot shift offset for partition $topicPartition since there is no current committed offset")).offset - (topicPartition, currentOffset + shiftBy) - }.toMap - checkOffsetsRange(requestedOffsets).map { - case (topicPartition, newOffset) => (topicPartition, new OffsetAndMetadata(newOffset)) - } - } else if (opts.options.has(opts.resetToDatetimeOpt)) { - val timestamp = Utils.getDateTime(opts.options.valueOf(opts.resetToDatetimeOpt)) - val logTimestampOffsets = getLogTimestampOffsets(partitionsToReset, timestamp) - partitionsToReset.map { topicPartition => - val logTimestampOffset = logTimestampOffsets.get(topicPartition) - logTimestampOffset match { - case Some(LogOffsetResult.LogOffset(offset)) => (topicPartition, new OffsetAndMetadata(offset)) - case _ => ToolsUtils.printUsageAndExit(opts.parser, s"Error getting offset by timestamp of topic partition: $topicPartition") - } - }.toMap - } else if (opts.options.has(opts.resetByDurationOpt)) { - val duration = opts.options.valueOf(opts.resetByDurationOpt) - val durationParsed = Duration.parse(duration) - val now = Instant.now() - durationParsed.negated().addTo(now) - val timestamp = now.minus(durationParsed).toEpochMilli - val logTimestampOffsets = getLogTimestampOffsets(partitionsToReset, timestamp) - partitionsToReset.map { topicPartition => - val logTimestampOffset = logTimestampOffsets.get(topicPartition) - logTimestampOffset match { - case Some(LogOffsetResult.LogOffset(offset)) => (topicPartition, new OffsetAndMetadata(offset)) - case _ => ToolsUtils.printUsageAndExit(opts.parser, s"Error getting offset by timestamp of topic partition: $topicPartition") - } - }.toMap - } else if (resetPlanFromFile.isDefined) { - resetPlanFromFile.map(resetPlan => resetPlan.get(groupId).map { resetPlanForGroup => - val requestedOffsets = resetPlanForGroup.keySet.map { topicPartition => - topicPartition -> resetPlanForGroup(topicPartition).offset - }.toMap - checkOffsetsRange(requestedOffsets).map { - case (topicPartition, newOffset) => (topicPartition, new OffsetAndMetadata(newOffset)) - } - } match { - case Some(resetPlanForGroup) => resetPlanForGroup - case None => - printError(s"No reset plan for group $groupId found") - Map[TopicPartition, OffsetAndMetadata]() - }).getOrElse(Map.empty) - } else if (opts.options.has(opts.resetToCurrentOpt)) { - val currentCommittedOffsets = getCommittedOffsets(groupId) - val (partitionsToResetWithCommittedOffset, partitionsToResetWithoutCommittedOffset) = - partitionsToReset.partition(currentCommittedOffsets.keySet.contains(_)) - - val preparedOffsetsForPartitionsWithCommittedOffset = partitionsToResetWithCommittedOffset.map { topicPartition => - (topicPartition, new OffsetAndMetadata(currentCommittedOffsets.get(topicPartition) match { - case Some(offset) => offset.offset - case None => throw new IllegalStateException(s"Expected a valid current offset for topic partition: $topicPartition") - })) - }.toMap - - val preparedOffsetsForPartitionsWithoutCommittedOffset = getLogEndOffsets(partitionsToResetWithoutCommittedOffset).map { - case (topicPartition, LogOffsetResult.LogOffset(offset)) => (topicPartition, new OffsetAndMetadata(offset)) - case (topicPartition, _) => ToolsUtils.printUsageAndExit(opts.parser, s"Error getting ending offset of topic partition: $topicPartition") - } - - preparedOffsetsForPartitionsWithCommittedOffset ++ preparedOffsetsForPartitionsWithoutCommittedOffset - } else { - ToolsUtils.printUsageAndExit(opts.parser, "Option '%s' requires one of the following scenarios: %s".format(opts.resetOffsetsOpt, opts.allResetOffsetScenarioOpts)) - } - } - - private def checkOffsetsRange(requestedOffsets: Map[TopicPartition, Long]) = { - val logStartOffsets = getLogStartOffsets(requestedOffsets.keySet.toSeq) - val logEndOffsets = getLogEndOffsets(requestedOffsets.keySet.toSeq) - requestedOffsets.map { case (topicPartition, offset) => (topicPartition, - logEndOffsets.get(topicPartition) match { - case Some(LogOffsetResult.LogOffset(endOffset)) if offset > endOffset => - warn(s"New offset ($offset) is higher than latest offset for topic partition $topicPartition. Value will be set to $endOffset") - endOffset - - case Some(_) => logStartOffsets.get(topicPartition) match { - case Some(LogOffsetResult.LogOffset(startOffset)) if offset < startOffset => - warn(s"New offset ($offset) is lower than earliest offset for topic partition $topicPartition. Value will be set to $startOffset") - startOffset - - case _ => offset - } - - case None => // the control should not reach here - throw new IllegalStateException(s"Unexpected non-existing offset value for topic partition $topicPartition") - }) - } - } - - def exportOffsetsToCsv(assignments: Map[String, Map[TopicPartition, OffsetAndMetadata]]): String = { - val isSingleGroupQuery = opts.options.valuesOf(opts.groupOpt).size() == 1 - val csvWriter = - if (isSingleGroupQuery) CsvUtils().writerFor[CsvRecordNoGroup] - else CsvUtils().writerFor[CsvRecordWithGroup] - val rows = assignments.flatMap { case (groupId, partitionInfo) => - partitionInfo.map { case (k: TopicPartition, v: OffsetAndMetadata) => - val csvRecord = - if (isSingleGroupQuery) CsvRecordNoGroup(k.topic, k.partition, v.offset) - else CsvRecordWithGroup(groupId, k.topic, k.partition, v.offset) - csvWriter.writeValueAsString(csvRecord) - } - } - rows.mkString("") - } - - def deleteGroups(): Map[String, Throwable] = { - val groupIds = - if (opts.options.has(opts.allGroupsOpt)) listConsumerGroups() - else opts.options.valuesOf(opts.groupOpt).asScala - - val groupsToDelete = adminClient.deleteConsumerGroups( - groupIds.asJava, - withTimeoutMs(new DeleteConsumerGroupsOptions) - ).deletedGroups().asScala - - val result = groupsToDelete.map { case (g, f) => - Try(f.get) match { - case Success(_) => g -> null - case Failure(e) => g -> e - } - } - - val (success, failed) = result.partition { - case (_, error) => error == null - } - - if (failed.isEmpty) { - println(s"Deletion of requested consumer groups (${success.keySet.mkString("'", "', '", "'")}) was successful.") - } - else { - printError("Deletion of some consumer groups failed:") - failed.foreach { - case (group, error) => println(s"* Group '$group' could not be deleted due to: ${error.toString}") - } - if (success.nonEmpty) - println(s"\nThese consumer groups were deleted successfully: ${success.keySet.mkString("'", "', '", "'")}") - } - - result.toMap - } - } - - sealed trait LogOffsetResult - - private object LogOffsetResult { - case class LogOffset(value: Long) extends LogOffsetResult - case object Unknown extends LogOffsetResult - case object Ignore extends LogOffsetResult - } - - class ConsumerGroupCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) { - val BootstrapServerDoc = "REQUIRED: The server(s) to connect to." - private val GroupDoc = "The consumer group we wish to act on." - private 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. " + - "Reset-offsets also supports multiple topic inputs." - private val AllTopicsDoc = "Consider all topics assigned to a group in the `reset-offsets` process." - private val ListDoc = "List all consumer groups." - private val DescribeDoc = "Describe consumer group and list offset lag (number of messages not yet processed) related to given group." - private val AllGroupsDoc = "Apply to all consumer groups." - val nl: String = System.getProperty("line.separator") - private val DeleteDoc = "Pass in groups to delete topic partition offsets and ownership information " + - "over the entire consumer group. For instance --group g1 --group g2" - private 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)." - val CommandConfigDoc: String = "Property file containing configs to be passed to Admin Client and Consumer." - private val ResetOffsetsDoc = "Reset offsets of consumer group. Supports one consumer group at the time, and instances should be inactive" + nl + - "Has 2 execution options: --dry-run (the default) to plan which offsets to reset, and --execute to update the offsets. " + - "Additionally, the --export option is used to export the results to a CSV format." + nl + - "You must choose one of the following reset specifications: --to-datetime, --by-duration, --to-earliest, " + - "--to-latest, --shift-by, --from-file, --to-current, --to-offset." + nl + - "To define the scope use --all-topics or --topic. One scope must be specified unless you use '--from-file'." - private val DryRunDoc = "Only show results without executing changes on Consumer Groups. Supported operations: reset-offsets." - private val ExecuteDoc = "Execute operation. Supported operations: reset-offsets." - private val ExportDoc = "Export operation execution to a CSV file. Supported operations: reset-offsets." - private val ResetToOffsetDoc = "Reset offsets to a specific offset." - private val ResetFromFileDoc = "Reset offsets to values defined in CSV file." - private val ResetToDatetimeDoc = "Reset offsets to offset from datetime. Format: 'YYYY-MM-DDTHH:mm:SS.sss'" - private val ResetByDurationDoc = "Reset offsets to offset by duration from current timestamp. Format: 'PnDTnHnMnS'" - private val ResetToEarliestDoc = "Reset offsets to earliest offset." - private val ResetToLatestDoc = "Reset offsets to latest offset." - private val ResetToCurrentDoc = "Reset offsets to current offset." - private val ResetShiftByDoc = "Reset offsets shifting current offset by 'n', where 'n' can be positive or negative." - private val MembersDoc = "Describe members of the group. This option may be used with '--describe' and '--bootstrap-server' options only." + nl + - "Example: --bootstrap-server localhost:9092 --describe --group group1 --members" - private val VerboseDoc = "Provide additional information, if any, when describing the group. This option may be used " + - "with '--offsets'/'--members'/'--state' and '--bootstrap-server' options only." + nl + "Example: --bootstrap-server localhost:9092 --describe --group group1 --members --verbose" - private val OffsetsDoc = "Describe the group and list all topic partitions in the group along with their offset lag. " + - "This is the default sub-action of and may be used with '--describe' and '--bootstrap-server' options only." + nl + - "Example: --bootstrap-server localhost:9092 --describe --group group1 --offsets" - private val StateDoc = "When specified with '--describe', includes the state of the group." + nl + - "Example: --bootstrap-server localhost:9092 --describe --group group1 --state" + nl + - "When specified with '--list', it displays the state of all groups. It can also be used to list groups with specific states." + nl + - "Example: --bootstrap-server localhost:9092 --list --state stable,empty" + nl + - "This option may be used with '--describe', '--list' and '--bootstrap-server' options only." - private val DeleteOffsetsDoc = "Delete offsets of consumer group. Supports one consumer group at the time, and multiple topics." - - val bootstrapServerOpt: OptionSpec[String] = parser.accepts("bootstrap-server", BootstrapServerDoc) - .withRequiredArg - .describedAs("server to connect to") - .ofType(classOf[String]) - val groupOpt: OptionSpec[String] = parser.accepts("group", GroupDoc) - .withRequiredArg - .describedAs("consumer group") - .ofType(classOf[String]) - val topicOpt: OptionSpec[String] = parser.accepts("topic", TopicDoc) - .withRequiredArg - .describedAs("topic") - .ofType(classOf[String]) - val allTopicsOpt: OptionSpecBuilder = parser.accepts("all-topics", AllTopicsDoc) - val listOpt: OptionSpecBuilder = parser.accepts("list", ListDoc) - val describeOpt: OptionSpecBuilder = parser.accepts("describe", DescribeDoc) - val allGroupsOpt: OptionSpecBuilder = parser.accepts("all-groups", AllGroupsDoc) - val deleteOpt: OptionSpecBuilder = parser.accepts("delete", DeleteDoc) - val timeoutMsOpt: OptionSpec[Long] = parser.accepts("timeout", TimeoutMsDoc) - .withRequiredArg - .describedAs("timeout (ms)") - .ofType(classOf[Long]) - .defaultsTo(5000) - val commandConfigOpt: OptionSpec[String] = parser.accepts("command-config", CommandConfigDoc) - .withRequiredArg - .describedAs("command config property file") - .ofType(classOf[String]) - val resetOffsetsOpt: OptionSpecBuilder = parser.accepts("reset-offsets", ResetOffsetsDoc) - val deleteOffsetsOpt: OptionSpecBuilder = parser.accepts("delete-offsets", DeleteOffsetsDoc) - val dryRunOpt: OptionSpecBuilder = parser.accepts("dry-run", DryRunDoc) - val executeOpt: OptionSpecBuilder = parser.accepts("execute", ExecuteDoc) - val exportOpt: OptionSpecBuilder = parser.accepts("export", ExportDoc) - val resetToOffsetOpt: OptionSpec[Long] = parser.accepts("to-offset", ResetToOffsetDoc) - .withRequiredArg() - .describedAs("offset") - .ofType(classOf[Long]) - val resetFromFileOpt: OptionSpec[String] = parser.accepts("from-file", ResetFromFileDoc) - .withRequiredArg() - .describedAs("path to CSV file") - .ofType(classOf[String]) - val resetToDatetimeOpt: OptionSpec[String] = parser.accepts("to-datetime", ResetToDatetimeDoc) - .withRequiredArg() - .describedAs("datetime") - .ofType(classOf[String]) - val resetByDurationOpt: OptionSpec[String] = parser.accepts("by-duration", ResetByDurationDoc) - .withRequiredArg() - .describedAs("duration") - .ofType(classOf[String]) - val resetToEarliestOpt: OptionSpecBuilder = parser.accepts("to-earliest", ResetToEarliestDoc) - val resetToLatestOpt: OptionSpecBuilder = parser.accepts("to-latest", ResetToLatestDoc) - val resetToCurrentOpt: OptionSpecBuilder = parser.accepts("to-current", ResetToCurrentDoc) - val resetShiftByOpt: OptionSpec[Long] = parser.accepts("shift-by", ResetShiftByDoc) - .withRequiredArg() - .describedAs("number-of-offsets") - .ofType(classOf[Long]) - val membersOpt: OptionSpecBuilder = parser.accepts("members", MembersDoc) - .availableIf(describeOpt) - val verboseOpt: OptionSpecBuilder = parser.accepts("verbose", VerboseDoc) - .availableIf(describeOpt) - val offsetsOpt: OptionSpecBuilder = parser.accepts("offsets", OffsetsDoc) - .availableIf(describeOpt) - val stateOpt: OptionSpec[String] = parser.accepts("state", StateDoc) - .availableIf(describeOpt, listOpt) - .withOptionalArg() - .ofType(classOf[String]) - - options = parser.parse(args : _*) - - private val allGroupSelectionScopeOpts = immutable.Set[OptionSpec[_]](groupOpt, allGroupsOpt) - private val allConsumerGroupLevelOpts = immutable.Set[OptionSpec[_]](listOpt, describeOpt, deleteOpt, resetOffsetsOpt) - val allResetOffsetScenarioOpts: Set[OptionSpec[_]] = immutable.Set[OptionSpec[_]](resetToOffsetOpt, resetShiftByOpt, - resetToDatetimeOpt, resetByDurationOpt, resetToEarliestOpt, resetToLatestOpt, resetToCurrentOpt, resetFromFileOpt) - private val allDeleteOffsetsOpts = immutable.Set[OptionSpec[_]](groupOpt, topicOpt) - - def checkArgs(): Unit = { - - CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt) - - if (options.has(describeOpt)) { - if (!options.has(groupOpt) && !options.has(allGroupsOpt)) - CommandLineUtils.printUsageAndExit(parser, - s"Option $describeOpt takes one of these options: ${allGroupSelectionScopeOpts.mkString(", ")}") - val mutuallyExclusiveOpts: Set[OptionSpec[_]] = Set(membersOpt, offsetsOpt, stateOpt) - if (mutuallyExclusiveOpts.toList.map(o => if (options.has(o)) 1 else 0).sum > 1) { - CommandLineUtils.printUsageAndExit(parser, - s"Option $describeOpt takes at most one of these options: ${mutuallyExclusiveOpts.mkString(", ")}") - } - if (options.has(stateOpt) && options.valueOf(stateOpt) != null) - CommandLineUtils.printUsageAndExit(parser, - s"Option $describeOpt does not take a value for $stateOpt") - } else { - if (options.has(timeoutMsOpt)) - debug(s"Option $timeoutMsOpt is applicable only when $describeOpt is used.") - } - - if (options.has(deleteOpt)) { - if (!options.has(groupOpt) && !options.has(allGroupsOpt)) - CommandLineUtils.printUsageAndExit(parser, - s"Option $deleteOpt takes one of these options: ${allGroupSelectionScopeOpts.mkString(", ")}") - if (options.has(topicOpt)) - CommandLineUtils.printUsageAndExit(parser, s"The consumer does not support topic-specific offset " + - "deletion from a consumer group.") - } - - if (options.has(deleteOffsetsOpt)) { - if (!options.has(groupOpt) || !options.has(topicOpt)) - CommandLineUtils.printUsageAndExit(parser, - s"Option $deleteOffsetsOpt takes the following options: ${allDeleteOffsetsOpts.mkString(", ")}") - } - - if (options.has(resetOffsetsOpt)) { - if (options.has(dryRunOpt) && options.has(executeOpt)) - CommandLineUtils.printUsageAndExit(parser, s"Option $resetOffsetsOpt only accepts one of $executeOpt and $dryRunOpt") - - if (!options.has(dryRunOpt) && !options.has(executeOpt)) { - Console.err.println("WARN: No action will be performed as the --execute option is missing." + - "In a future major release, the default behavior of this command will be to prompt the user before " + - "executing the reset rather than doing a dry run. You should add the --dry-run option explicitly " + - "if you are scripting this command and want to keep the current default behavior without prompting.") - } - - if (!options.has(groupOpt) && !options.has(allGroupsOpt)) - CommandLineUtils.printUsageAndExit(parser, - s"Option $resetOffsetsOpt takes one of these options: ${allGroupSelectionScopeOpts.mkString(", ")}") - CommandLineUtils.checkInvalidArgs(parser, options, resetToOffsetOpt, (allResetOffsetScenarioOpts - resetToOffsetOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetToDatetimeOpt, (allResetOffsetScenarioOpts - resetToDatetimeOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetByDurationOpt, (allResetOffsetScenarioOpts - resetByDurationOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetToEarliestOpt, (allResetOffsetScenarioOpts - resetToEarliestOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetToLatestOpt, (allResetOffsetScenarioOpts - resetToLatestOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetToCurrentOpt, (allResetOffsetScenarioOpts - resetToCurrentOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetShiftByOpt, (allResetOffsetScenarioOpts - resetShiftByOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, resetFromFileOpt, (allResetOffsetScenarioOpts - resetFromFileOpt).asJava) - } - - CommandLineUtils.checkInvalidArgs(parser, options, groupOpt, (allGroupSelectionScopeOpts - groupOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, groupOpt, (allConsumerGroupLevelOpts - describeOpt - deleteOpt - resetOffsetsOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, topicOpt, (allConsumerGroupLevelOpts - deleteOpt - resetOffsetsOpt).asJava ) - } - } -} diff --git a/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala b/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala index 59af648c21..b912881847 100644 --- a/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala +++ b/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala @@ -17,7 +17,7 @@ package kafka.coordinator.group import kafka.cluster.PartitionListener -import kafka.server.{ActionQueue, ReplicaManager, RequestLocal} +import kafka.server.{ActionQueue, ReplicaManager, RequestLocal, defaultError, genericError} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.RecordTooLargeException import org.apache.kafka.common.protocol.Errors @@ -188,8 +188,10 @@ class CoordinatorPartitionWriter[T]( tp: TopicPartition, transactionalId: String, producerId: Long, - producerEpoch: Short + producerEpoch: Short, + apiVersion: Short ): CompletableFuture[VerificationGuard] = { + val supportedOperation = if (apiVersion >= 4) genericError else defaultError val future = new CompletableFuture[VerificationGuard]() replicaManager.maybeStartTransactionVerificationForPartition( topicPartition = tp, @@ -204,7 +206,8 @@ class CoordinatorPartitionWriter[T]( } else { future.complete(verificationGuard) } - } + }, + supportedOperation ) future } diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala index 328dfbf29d..8c578411d3 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala @@ -905,7 +905,8 @@ private[group] class GroupCoordinator( generationId: Int, offsetMetadata: immutable.Map[TopicIdPartition, OffsetAndMetadata], responseCallback: immutable.Map[TopicIdPartition, Errors] => Unit, - requestLocal: RequestLocal = RequestLocal.NoCaching): Unit = { + requestLocal: RequestLocal = RequestLocal.NoCaching, + apiVersion: Short): Unit = { validateGroupStatus(groupId, ApiKeys.TXN_OFFSET_COMMIT) match { case Some(error) => responseCallback(offsetMetadata.map { case (k, _) => k -> error }) case None => @@ -929,6 +930,7 @@ private[group] class GroupCoordinator( } } + val supportedOperation = if (apiVersion >= 4) genericError else defaultError // AutoMQ inject start val replicaManager = groupManager.replicaManager val verification = replicaManager.verify(transactionalId, producerId) @@ -945,7 +947,8 @@ private[group] class GroupCoordinator( KafkaRequestHandler.wrapAsyncCallback( replicaManager.verifyTransactionCallbackWrapper(verification, postVerificationCallback), requestLocal - ) + ), + supportedOperation ) } val hasInflight = replicaManager.checkWaitingTransaction(verification, task) @@ -954,6 +957,7 @@ private[group] class GroupCoordinator( } task() // AutoMQ inject end + } } diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinatorAdapter.scala b/core/src/main/scala/kafka/coordinator/group/GroupCoordinatorAdapter.scala index 6b3cf072c8..53d7c9b5e9 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinatorAdapter.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinatorAdapter.scala @@ -476,7 +476,8 @@ private[group] class GroupCoordinatorAdapter( request.generationId, partitions.toMap, callback, - RequestLocal(bufferSupplier) + RequestLocal(bufferSupplier), + context.apiVersion() ) future diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index d59accbe99..13d9cd28bb 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -1370,6 +1370,14 @@ object GroupMetadataManager { def maybeConvertOffsetCommitError(error: Errors) : Errors = { error match { + case Errors.NETWORK_EXCEPTION => + // When committing offsets transactionally, we now verify the transaction with the + // transaction coordinator. Verification can fail with `NETWORK_EXCEPTION`, a retriable + // error which older clients may not expect and retry correctly. We translate the error to + // `COORDINATOR_LOAD_IN_PROGRESS` because it causes clients to retry the request without an + // unnecessary coordinator lookup. + Errors.COORDINATOR_LOAD_IN_PROGRESS + case Errors.UNKNOWN_TOPIC_OR_PARTITION | Errors.NOT_ENOUGH_REPLICAS | Errors.NOT_ENOUGH_REPLICAS_AFTER_APPEND => diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index c7f5a16b06..4d9ba7fa55 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -369,7 +369,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig, if (txnMetadata.topicPartitions.contains(part)) (part, Errors.NONE) else - (part, Errors.INVALID_TXN_STATE) + (part, Errors.ABORTABLE_TRANSACTION) }.toMap) } } diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala index 5a72554c74..068dff4cca 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala @@ -109,23 +109,43 @@ object TransactionMarkerChannelManager { } -class TxnMarkerQueue(@volatile var destination: Node) { +class TxnMarkerQueue(@volatile var destination: Node) extends Logging { // keep track of the requests per txn topic partition so we can easily clear the queue // during partition emigration - private val markersPerTxnTopicPartition = new ConcurrentHashMap[Int, BlockingQueue[TxnIdAndMarkerEntry]]().asScala + private val markersPerTxnTopicPartition = new ConcurrentHashMap[Int, BlockingQueue[PendingCompleteTxnAndMarkerEntry]]().asScala - def removeMarkersForTxnTopicPartition(partition: Int): Option[BlockingQueue[TxnIdAndMarkerEntry]] = { + def removeMarkersForTxnTopicPartition(partition: Int): Option[BlockingQueue[PendingCompleteTxnAndMarkerEntry]] = { markersPerTxnTopicPartition.remove(partition) } - def addMarkers(txnTopicPartition: Int, txnIdAndMarker: TxnIdAndMarkerEntry): Unit = { - val queue = CoreUtils.atomicGetOrUpdate(markersPerTxnTopicPartition, txnTopicPartition, - new LinkedBlockingQueue[TxnIdAndMarkerEntry]()) - queue.add(txnIdAndMarker) + def addMarkers(txnTopicPartition: Int, pendingCompleteTxnAndMarker: PendingCompleteTxnAndMarkerEntry): Unit = { + val queue = CoreUtils.atomicGetOrUpdate(markersPerTxnTopicPartition, txnTopicPartition, { + // Note that this may get called more than once if threads have a close race while adding new queue. + info(s"Creating new marker queue for txn partition $txnTopicPartition to destination broker ${destination.id}") + new LinkedBlockingQueue[PendingCompleteTxnAndMarkerEntry]() + }) + queue.add(pendingCompleteTxnAndMarker) + + if (markersPerTxnTopicPartition.get(txnTopicPartition).orNull != queue) { + // This could happen if the queue got removed concurrently. + // Note that it could create an unexpected state when the queue is removed from + // removeMarkersForTxnTopicPartition, we could have: + // + // 1. [addMarkers] Retrieve queue. + // 2. [removeMarkersForTxnTopicPartition] Remove queue. + // 3. [removeMarkersForTxnTopicPartition] Iterate over queue, but not removeMarkersForTxn because queue is empty. + // 4. [addMarkers] Add markers to the queue. + // + // Now we've effectively removed the markers while transactionsWithPendingMarkers has an entry. + // + // While this could lead to an orphan entry in transactionsWithPendingMarkers, sending new markers + // will fix the state, so it shouldn't impact the state machine operation. + warn(s"Added $pendingCompleteTxnAndMarker to dead queue for txn partition $txnTopicPartition to destination broker ${destination.id}") + } } - def forEachTxnTopicPartition[B](f:(Int, BlockingQueue[TxnIdAndMarkerEntry]) => B): Unit = + def forEachTxnTopicPartition[B](f:(Int, BlockingQueue[PendingCompleteTxnAndMarkerEntry]) => B): Unit = markersPerTxnTopicPartition.forKeyValue { (partition, queue) => if (!queue.isEmpty) f(partition, queue) } @@ -187,17 +207,21 @@ class TransactionMarkerChannelManager( // visible for testing private[transaction] def queueForUnknownBroker = markersQueueForUnknownBroker - private[transaction] def addMarkersForBroker(broker: Node, txnTopicPartition: Int, txnIdAndMarker: TxnIdAndMarkerEntry): Unit = { + private[transaction] def addMarkersForBroker(broker: Node, txnTopicPartition: Int, pendingCompleteTxnAndMarker: PendingCompleteTxnAndMarkerEntry): Unit = { val brokerId = broker.id // we do not synchronize on the update of the broker node with the enqueuing, // since even if there is a race condition we will just retry - val brokerRequestQueue = CoreUtils.atomicGetOrUpdate(markersQueuePerBroker, brokerId, - new TxnMarkerQueue(broker)) + val brokerRequestQueue = CoreUtils.atomicGetOrUpdate(markersQueuePerBroker, brokerId, { + // Note that this may get called more than once if threads have a close race while adding new queue. + info(s"Creating new marker queue map to destination broker $brokerId") + new TxnMarkerQueue(broker) + }) brokerRequestQueue.destination = broker - brokerRequestQueue.addMarkers(txnTopicPartition, txnIdAndMarker) + brokerRequestQueue.addMarkers(txnTopicPartition, pendingCompleteTxnAndMarker) - trace(s"Added marker ${txnIdAndMarker.txnMarkerEntry} for transactional id ${txnIdAndMarker.txnId} to destination broker $brokerId") + trace(s"Added marker ${pendingCompleteTxnAndMarker.txnMarkerEntry} for transactional id" + + s" ${pendingCompleteTxnAndMarker.pendingCompleteTxn.transactionalId} to destination broker $brokerId") } private def retryLogAppends(): Unit = { @@ -211,29 +235,28 @@ class TransactionMarkerChannelManager( override def generateRequests(): util.Collection[RequestAndCompletionHandler] = { retryLogAppends() - val txnIdAndMarkerEntries: util.List[TxnIdAndMarkerEntry] = new util.ArrayList[TxnIdAndMarkerEntry]() + val pendingCompleteTxnAndMarkerEntries = new util.ArrayList[PendingCompleteTxnAndMarkerEntry]() markersQueueForUnknownBroker.forEachTxnTopicPartition { case (_, queue) => - queue.drainTo(txnIdAndMarkerEntries) + queue.drainTo(pendingCompleteTxnAndMarkerEntries) } - for (txnIdAndMarker: TxnIdAndMarkerEntry <- txnIdAndMarkerEntries.asScala) { - val transactionalId = txnIdAndMarker.txnId - val producerId = txnIdAndMarker.txnMarkerEntry.producerId - val producerEpoch = txnIdAndMarker.txnMarkerEntry.producerEpoch - val txnResult = txnIdAndMarker.txnMarkerEntry.transactionResult - val coordinatorEpoch = txnIdAndMarker.txnMarkerEntry.coordinatorEpoch - val topicPartitions = txnIdAndMarker.txnMarkerEntry.partitions.asScala.toSet + for (pendingCompleteTxnAndMarker: PendingCompleteTxnAndMarkerEntry <- pendingCompleteTxnAndMarkerEntries.asScala) { + val producerId = pendingCompleteTxnAndMarker.txnMarkerEntry.producerId + val producerEpoch = pendingCompleteTxnAndMarker.txnMarkerEntry.producerEpoch + val txnResult = pendingCompleteTxnAndMarker.txnMarkerEntry.transactionResult + val pendingCompleteTxn = pendingCompleteTxnAndMarker.pendingCompleteTxn + val topicPartitions = pendingCompleteTxnAndMarker.txnMarkerEntry.partitions.asScala.toSet - addTxnMarkersToBrokerQueue(transactionalId, producerId, producerEpoch, txnResult, coordinatorEpoch, topicPartitions) + addTxnMarkersToBrokerQueue(producerId, producerEpoch, txnResult, pendingCompleteTxn, topicPartitions) } val currentTimeMs = time.milliseconds() markersQueuePerBroker.values.map { brokerRequestQueue => - val txnIdAndMarkerEntries = new util.ArrayList[TxnIdAndMarkerEntry]() + val pendingCompleteTxnAndMarkerEntries = new util.ArrayList[PendingCompleteTxnAndMarkerEntry]() brokerRequestQueue.forEachTxnTopicPartition { case (_, queue) => - queue.drainTo(txnIdAndMarkerEntries) + queue.drainTo(pendingCompleteTxnAndMarkerEntries) } - (brokerRequestQueue.destination, txnIdAndMarkerEntries) + (brokerRequestQueue.destination, pendingCompleteTxnAndMarkerEntries) }.filter { case (_, entries) => !entries.isEmpty }.map { case (node, entries) => val markersToSend = entries.asScala.map(_.txnMarkerEntry).asJava val requestCompletionHandler = new TransactionMarkerRequestCompletionHandler(node.id, txnStateManager, this, entries) @@ -300,9 +323,12 @@ class TransactionMarkerChannelManager( txnMetadata, newMetadata) - transactionsWithPendingMarkers.put(transactionalId, pendingCompleteTxn) - addTxnMarkersToBrokerQueue(transactionalId, txnMetadata.producerId, - txnMetadata.producerEpoch, txnResult, coordinatorEpoch, txnMetadata.topicPartitions.toSet) + val prev = transactionsWithPendingMarkers.put(transactionalId, pendingCompleteTxn) + if (prev != null) { + info(s"Replaced an existing pending complete txn $prev with $pendingCompleteTxn while adding markers to send.") + } + addTxnMarkersToBrokerQueue(txnMetadata.producerId, + txnMetadata.producerEpoch, txnResult, pendingCompleteTxn, txnMetadata.topicPartitions.toSet) maybeWriteTxnCompletion(transactionalId) } @@ -354,41 +380,42 @@ class TransactionMarkerChannelManager( txnLogAppend.newMetadata, appendCallback, _ == Errors.COORDINATOR_NOT_AVAILABLE, RequestLocal.NoCaching) } - def addTxnMarkersToBrokerQueue(transactionalId: String, - producerId: Long, + def addTxnMarkersToBrokerQueue(producerId: Long, producerEpoch: Short, result: TransactionResult, - coordinatorEpoch: Int, + pendingCompleteTxn: PendingCompleteTxn, topicPartitions: immutable.Set[TopicPartition]): Unit = { - val txnTopicPartition = txnStateManager.partitionFor(transactionalId) + val txnTopicPartition = txnStateManager.partitionFor(pendingCompleteTxn.transactionalId) val partitionsByDestination: immutable.Map[Option[Node], immutable.Set[TopicPartition]] = topicPartitions.groupBy { topicPartition: TopicPartition => metadataCache.getPartitionLeaderEndpoint(topicPartition.topic, topicPartition.partition, interBrokerListenerName) } + val coordinatorEpoch = pendingCompleteTxn.coordinatorEpoch for ((broker: Option[Node], topicPartitions: immutable.Set[TopicPartition]) <- partitionsByDestination) { broker match { case Some(brokerNode) => val marker = new TxnMarkerEntry(producerId, producerEpoch, coordinatorEpoch, result, topicPartitions.toList.asJava) - val txnIdAndMarker = TxnIdAndMarkerEntry(transactionalId, marker) + val pendingCompleteTxnAndMarker = PendingCompleteTxnAndMarkerEntry(pendingCompleteTxn, marker) if (brokerNode == Node.noNode) { // if the leader of the partition is known but node not available, put it into an unknown broker queue // and let the sender thread to look for its broker and migrate them later - markersQueueForUnknownBroker.addMarkers(txnTopicPartition, txnIdAndMarker) + markersQueueForUnknownBroker.addMarkers(txnTopicPartition, pendingCompleteTxnAndMarker) } else { - addMarkersForBroker(brokerNode, txnTopicPartition, txnIdAndMarker) + addMarkersForBroker(brokerNode, txnTopicPartition, pendingCompleteTxnAndMarker) } case None => + val transactionalId = pendingCompleteTxn.transactionalId txnStateManager.getTransactionState(transactionalId) match { case Left(error) => info(s"Encountered $error trying to fetch transaction metadata for $transactionalId with coordinator epoch $coordinatorEpoch; cancel sending markers to its partition leaders") - transactionsWithPendingMarkers.remove(transactionalId) + transactionsWithPendingMarkers.remove(transactionalId, pendingCompleteTxn) case Right(Some(epochAndMetadata)) => if (epochAndMetadata.coordinatorEpoch != coordinatorEpoch) { info(s"The cached metadata has changed to $epochAndMetadata (old coordinator epoch is $coordinatorEpoch) since preparing to send markers; cancel sending markers to its partition leaders") - transactionsWithPendingMarkers.remove(transactionalId) + transactionsWithPendingMarkers.remove(transactionalId, pendingCompleteTxn) } else { // if the leader of the partition is unknown, skip sending the txn marker since // the partition is likely to be deleted already @@ -419,25 +446,34 @@ class TransactionMarkerChannelManager( def removeMarkersForTxnTopicPartition(txnTopicPartitionId: Int): Unit = { markersQueueForUnknownBroker.removeMarkersForTxnTopicPartition(txnTopicPartitionId).foreach { queue => - for (entry: TxnIdAndMarkerEntry <- queue.asScala) - removeMarkersForTxnId(entry.txnId) + for (entry <- queue.asScala) { + info(s"Removing $entry for txn partition $txnTopicPartitionId to destination broker -1") + removeMarkersForTxn(entry.pendingCompleteTxn) + } } - markersQueuePerBroker.foreach { case(_, brokerQueue) => + markersQueuePerBroker.foreach { case(brokerId, brokerQueue) => brokerQueue.removeMarkersForTxnTopicPartition(txnTopicPartitionId).foreach { queue => - for (entry: TxnIdAndMarkerEntry <- queue.asScala) - removeMarkersForTxnId(entry.txnId) + for (entry <- queue.asScala) { + info(s"Removing $entry for txn partition $txnTopicPartitionId to destination broker $brokerId") + removeMarkersForTxn(entry.pendingCompleteTxn) + } } } } - def removeMarkersForTxnId(transactionalId: String): Unit = { - transactionsWithPendingMarkers.remove(transactionalId) + def removeMarkersForTxn(pendingCompleteTxn: PendingCompleteTxn): Unit = { + val transactionalId = pendingCompleteTxn.transactionalId + val removed = transactionsWithPendingMarkers.remove(transactionalId, pendingCompleteTxn) + if (!removed) { + val current = transactionsWithPendingMarkers.get(transactionalId) + if (current != null) { + info(s"Failed to remove pending marker entry $current trying to remove $pendingCompleteTxn") + } + } } } -case class TxnIdAndMarkerEntry(txnId: String, txnMarkerEntry: TxnMarkerEntry) - case class PendingCompleteTxn(transactionalId: String, coordinatorEpoch: Int, txnMetadata: TransactionMetadata, @@ -451,3 +487,5 @@ case class PendingCompleteTxn(transactionalId: String, s"newMetadata=$newMetadata)" } } + +case class PendingCompleteTxnAndMarkerEntry(pendingCompleteTxn: PendingCompleteTxn, txnMarkerEntry: TxnMarkerEntry) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandler.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandler.scala index 7a59139b17..d95dabab6c 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandler.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandler.scala @@ -29,7 +29,7 @@ import scala.jdk.CollectionConverters._ class TransactionMarkerRequestCompletionHandler(brokerId: Int, txnStateManager: TransactionStateManager, txnMarkerChannelManager: TransactionMarkerChannelManager, - txnIdAndMarkerEntries: java.util.List[TxnIdAndMarkerEntry]) extends RequestCompletionHandler with Logging { + pendingCompleteTxnAndMarkerEntries: java.util.List[PendingCompleteTxnAndMarkerEntry]) extends RequestCompletionHandler with Logging { this.logIdent = "[Transaction Marker Request Completion Handler " + brokerId + "]: " @@ -39,22 +39,23 @@ class TransactionMarkerRequestCompletionHandler(brokerId: Int, if (response.wasDisconnected) { trace(s"Cancelled request with header $requestHeader due to node ${response.destination} being disconnected") - for (txnIdAndMarker <- txnIdAndMarkerEntries.asScala) { - val transactionalId = txnIdAndMarker.txnId - val txnMarker = txnIdAndMarker.txnMarkerEntry + for (pendingCompleteTxnAndMarker <- pendingCompleteTxnAndMarkerEntries.asScala) { + val pendingCompleteTxn = pendingCompleteTxnAndMarker.pendingCompleteTxn + val transactionalId = pendingCompleteTxn.transactionalId + val txnMarker = pendingCompleteTxnAndMarker.txnMarkerEntry txnStateManager.getTransactionState(transactionalId) match { case Left(Errors.NOT_COORDINATOR) => info(s"I am no longer the coordinator for $transactionalId; cancel sending transaction markers $txnMarker to the brokers") - txnMarkerChannelManager.removeMarkersForTxnId(transactionalId) + txnMarkerChannelManager.removeMarkersForTxn(pendingCompleteTxn) case Left(Errors.COORDINATOR_LOAD_IN_PROGRESS) => info(s"I am loading the transaction partition that contains $transactionalId which means the current markers have to be obsoleted; " + s"cancel sending transaction markers $txnMarker to the brokers") - txnMarkerChannelManager.removeMarkersForTxnId(transactionalId) + txnMarkerChannelManager.removeMarkersForTxn(pendingCompleteTxn) case Left(unexpectedError) => throw new IllegalStateException(s"Unhandled error $unexpectedError when fetching current transaction state") @@ -69,17 +70,16 @@ class TransactionMarkerRequestCompletionHandler(brokerId: Int, info(s"Transaction coordinator epoch for $transactionalId has changed from ${txnMarker.coordinatorEpoch} to " + s"${epochAndMetadata.coordinatorEpoch}; cancel sending transaction markers $txnMarker to the brokers") - txnMarkerChannelManager.removeMarkersForTxnId(transactionalId) + txnMarkerChannelManager.removeMarkersForTxn(pendingCompleteTxn) } else { // re-enqueue the markers with possibly new destination brokers trace(s"Re-enqueuing ${txnMarker.transactionResult} transaction markers for transactional id $transactionalId " + s"under coordinator epoch ${txnMarker.coordinatorEpoch}") - txnMarkerChannelManager.addTxnMarkersToBrokerQueue(transactionalId, - txnMarker.producerId, + txnMarkerChannelManager.addTxnMarkersToBrokerQueue(txnMarker.producerId, txnMarker.producerEpoch, txnMarker.transactionResult, - txnMarker.coordinatorEpoch, + pendingCompleteTxn, txnMarker.partitions.asScala.toSet) } } @@ -90,9 +90,10 @@ class TransactionMarkerRequestCompletionHandler(brokerId: Int, val writeTxnMarkerResponse = response.responseBody.asInstanceOf[WriteTxnMarkersResponse] val responseErrors = writeTxnMarkerResponse.errorsByProducerId - for (txnIdAndMarker <- txnIdAndMarkerEntries.asScala) { - val transactionalId = txnIdAndMarker.txnId - val txnMarker = txnIdAndMarker.txnMarkerEntry + for (pendingCompleteTxnAndMarker <- pendingCompleteTxnAndMarkerEntries.asScala) { + val pendingCompleteTxn = pendingCompleteTxnAndMarker.pendingCompleteTxn + val transactionalId = pendingCompleteTxn.transactionalId + val txnMarker = pendingCompleteTxnAndMarker.txnMarkerEntry val errors = responseErrors.get(txnMarker.producerId) if (errors == null) @@ -102,13 +103,13 @@ class TransactionMarkerRequestCompletionHandler(brokerId: Int, case Left(Errors.NOT_COORDINATOR) => info(s"I am no longer the coordinator for $transactionalId; cancel sending transaction markers $txnMarker to the brokers") - txnMarkerChannelManager.removeMarkersForTxnId(transactionalId) + txnMarkerChannelManager.removeMarkersForTxn(pendingCompleteTxn) case Left(Errors.COORDINATOR_LOAD_IN_PROGRESS) => info(s"I am loading the transaction partition that contains $transactionalId which means the current markers have to be obsoleted; " + s"cancel sending transaction markers $txnMarker to the brokers") - txnMarkerChannelManager.removeMarkersForTxnId(transactionalId) + txnMarkerChannelManager.removeMarkersForTxn(pendingCompleteTxn) case Left(unexpectedError) => throw new IllegalStateException(s"Unhandled error $unexpectedError when fetching current transaction state") @@ -127,7 +128,7 @@ class TransactionMarkerRequestCompletionHandler(brokerId: Int, info(s"Transaction coordinator epoch for $transactionalId has changed from ${txnMarker.coordinatorEpoch} to " + s"${epochAndMetadata.coordinatorEpoch}; cancel sending transaction markers $txnMarker to the brokers") - txnMarkerChannelManager.removeMarkersForTxnId(transactionalId) + txnMarkerChannelManager.removeMarkersForTxn(pendingCompleteTxn) abortSending = true } else { txnMetadata.inLock { @@ -161,7 +162,7 @@ class TransactionMarkerRequestCompletionHandler(brokerId: Int, info(s"Sending $transactionalId's transaction marker for partition $topicPartition has permanently failed with error ${error.exceptionName} " + s"with the current coordinator epoch ${epochAndMetadata.coordinatorEpoch}; cancel sending any more transaction markers $txnMarker to the brokers") - txnMarkerChannelManager.removeMarkersForTxnId(transactionalId) + txnMarkerChannelManager.removeMarkersForTxn(pendingCompleteTxn) abortSending = true case Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT | @@ -187,11 +188,10 @@ class TransactionMarkerRequestCompletionHandler(brokerId: Int, // re-enqueue with possible new leaders of the partitions txnMarkerChannelManager.addTxnMarkersToBrokerQueue( - transactionalId, txnMarker.producerId, txnMarker.producerEpoch, txnMarker.transactionResult, - txnMarker.coordinatorEpoch, + pendingCompleteTxn, retryPartitions.toSet) } else { txnMarkerChannelManager.maybeWriteTxnCompletion(transactionalId) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index fe441bb607..b6c97a5a25 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -562,6 +562,7 @@ class TransactionStateManager(brokerId: Int, loadingPartitions.remove(partitionAndLeaderEpoch) transactionsPendingForCompletion.foreach { txnTransitMetadata => + info(s"Sending txn markers for $txnTransitMetadata after loading partition $partitionId") sendTxnMarkers(txnTransitMetadata.coordinatorEpoch, txnTransitMetadata.result, txnTransitMetadata.txnMetadata, txnTransitMetadata.transitMetadata) } diff --git a/core/src/main/scala/kafka/log/LocalLog.scala b/core/src/main/scala/kafka/log/LocalLog.scala index 501d12b101..e36b96ca99 100644 --- a/core/src/main/scala/kafka/log/LocalLog.scala +++ b/core/src/main/scala/kafka/log/LocalLog.scala @@ -410,8 +410,8 @@ class LocalLog(@volatile protected var _dir: File, } } - private[log] def append(lastOffset: Long, largestTimestamp: Long, shallowOffsetOfMaxTimestamp: Long, records: MemoryRecords): Unit = { - segments.activeSegment.append(lastOffset, largestTimestamp, shallowOffsetOfMaxTimestamp, records) + private[log] def append(lastOffset: Long, largestTimestamp: Long, offsetOfMaxTimestamp: Long, records: MemoryRecords): Unit = { + segments.activeSegment.append(lastOffset, largestTimestamp, offsetOfMaxTimestamp, records) updateLogEndOffset(lastOffset + 1) } diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 71c92410f2..dfc0ea72de 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -500,13 +500,13 @@ class LogCleaner(initialConfig: CleanerConfig, object LogCleaner { val ReconfigurableConfigs: Set[String] = Set( - KafkaConfig.LogCleanerThreadsProp, - KafkaConfig.LogCleanerDedupeBufferSizeProp, - KafkaConfig.LogCleanerDedupeBufferLoadFactorProp, - KafkaConfig.LogCleanerIoBufferSizeProp, + CleanerConfig.LOG_CLEANER_THREADS_PROP, + CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, + CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, + CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, KafkaConfig.MessageMaxBytesProp, - KafkaConfig.LogCleanerIoMaxBytesPerSecondProp, - KafkaConfig.LogCleanerBackoffMsProp + CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, + CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP ) def cleanerConfig(config: KafkaConfig): CleanerConfig = { @@ -821,7 +821,7 @@ private[log] class Cleaner(val id: Int, val retained = MemoryRecords.readableRecords(outputBuffer) // it's OK not to hold the Log's lock in this case, because this segment is only accessed by other threads // after `Log.replaceSegments` (which acquires the lock) is called - dest.append(result.maxOffset, result.maxTimestamp, result.shallowOffsetOfMaxTimestamp, retained) + dest.append(result.maxOffset, result.maxTimestamp, result.offsetOfMaxTimestamp, retained) throttler.maybeThrottle(outputBuffer.limit()) } @@ -1231,7 +1231,7 @@ private[log] class Cleaner(val id: Int, val retained = MemoryRecords.readableRecords(outputBuffer) // it's OK not to hold the Log's lock in this case, because this segment is only accessed by other threads // after `Log.replaceSegments` (which acquires the lock) is called - dest.append(result.maxOffset, result.maxTimestamp, result.shallowOffsetOfMaxTimestamp, retained) + dest.append(result.maxOffset, result.maxTimestamp, result.offsetOfMaxTimestamp, retained) throttler.maybeThrottle(outputBuffer.limit()) } } diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index c312e70e07..0a7ca11356 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -901,7 +901,8 @@ class LogManager(logDirs: Seq[File], try { logStartOffsetCheckpoints.get(logDir).foreach { checkpoint => val logStartOffsets = logsToCheckpoint.collect { - case (tp, log) if log.logStartOffset > log.logSegments.asScala.head.baseOffset => tp -> log.logStartOffset + case (tp, log) if log.remoteLogEnabled() || log.logStartOffset > log.logSegments.asScala.head.baseOffset => + tp -> log.logStartOffset } checkpoint.write(logStartOffsets) } @@ -1671,7 +1672,7 @@ object LogManager { Option(newTopicsImage.getPartition(topicId, partitionId)) match { case Some(partition) => if (!partition.replicas.contains(brokerId)) { - info(s"Found stray log dir $log: the current replica assignment ${partition.replicas} " + + info(s"Found stray log dir $log: the current replica assignment ${partition.replicas.mkString("[", ", ", "]")} " + s"does not contain the local brokerId $brokerId.") Some(log.topicPartition) } else { diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index ec2c7b1c78..d904947f49 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -42,7 +42,7 @@ import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.util.Scheduler import org.apache.kafka.storage.internals.checkpoint.{LeaderEpochCheckpointFile, PartitionMetadataFile} import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache -import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, BatchMetadata, CompletedTxn, EpochEntry, FetchDataInfo, FetchIsolation, LastRecord, LeaderHwChange, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogValidator, ProducerAppendInfo, ProducerStateManager, ProducerStateManagerConfig, RollParams, VerificationGuard} +import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, BatchMetadata, CompletedTxn, FetchDataInfo, FetchIsolation, LastRecord, LeaderHwChange, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogValidator, ProducerAppendInfo, ProducerStateManager, ProducerStateManagerConfig, RollParams, VerificationGuard} import java.io.{File, IOException} import java.nio.file.{Files, Path} @@ -151,7 +151,9 @@ class UnifiedLog(@volatile var logStartOffset: Long, def localLogStartOffset(): Long = _localLogStartOffset // This is the offset(inclusive) until which segments are copied to the remote storage. - @volatile private var highestOffsetInRemoteStorage: Long = -1L + @volatile private[kafka] var _highestOffsetInRemoteStorage: Long = -1L + + def highestOffsetInRemoteStorage(): Long = _highestOffsetInRemoteStorage locally { def updateLocalLogStartOffset(offset: Long): Unit = { @@ -545,8 +547,8 @@ class UnifiedLog(@volatile var logStartOffset: Long, def updateHighestOffsetInRemoteStorage(offset: Long): Unit = { if (!remoteLogEnabled()) - warn(s"Unable to update the highest offset in remote storage with offset $offset since remote storage is not enabled. The existing highest offset is $highestOffsetInRemoteStorage.") - else if (offset > highestOffsetInRemoteStorage) highestOffsetInRemoteStorage = offset + warn(s"Unable to update the highest offset in remote storage with offset $offset since remote storage is not enabled. The existing highest offset is ${highestOffsetInRemoteStorage()}.") + else if (offset > highestOffsetInRemoteStorage()) _highestOffsetInRemoteStorage = offset } // Rebuild producer state until lastOffset. This method may be called from the recovery code path, and thus must be @@ -818,7 +820,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, validRecords = validateAndOffsetAssignResult.validatedRecords appendInfo.setMaxTimestamp(validateAndOffsetAssignResult.maxTimestampMs) - appendInfo.setOffsetOfMaxTimestamp(validateAndOffsetAssignResult.shallowOffsetOfMaxTimestampMs) + appendInfo.setOffsetOfMaxTimestamp(validateAndOffsetAssignResult.offsetOfMaxTimestampMs) appendInfo.setLastOffset(offset.value - 1) appendInfo.setRecordValidationStats(validateAndOffsetAssignResult.recordValidationStats) if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME) @@ -1285,7 +1287,6 @@ class UnifiedLog(@volatile var logStartOffset: Long, if (config.messageFormatVersion.isLessThan(IBP_0_10_0_IV0) && targetTimestamp != ListOffsetsRequest.EARLIEST_TIMESTAMP && - targetTimestamp != ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP && targetTimestamp != ListOffsetsRequest.LATEST_TIMESTAMP) throw new UnsupportedForMessageFormatException(s"Cannot search offsets based on timestamp because message format version " + s"for partition $topicPartition is ${config.messageFormatVersion} which is earlier than the minimum " + @@ -1306,18 +1307,39 @@ class UnifiedLog(@volatile var logStartOffset: Long, } else if (targetTimestamp == ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP) { val curLocalLogStartOffset = localLogStartOffset() - val earliestLocalLogEpochEntry = leaderEpochCache.asJava.flatMap(cache => { - val epoch = cache.epochForOffset(curLocalLogStartOffset) - if (epoch.isPresent) cache.epochEntry(epoch.getAsInt) else Optional.empty[EpochEntry]() - }) - - val epochOpt = if (earliestLocalLogEpochEntry.isPresent && earliestLocalLogEpochEntry.get().startOffset <= curLocalLogStartOffset) - Optional.of[Integer](earliestLocalLogEpochEntry.get().epoch) - else Optional.empty[Integer]() + val epochResult: Optional[Integer] = + if (leaderEpochCache.isDefined) { + val epochOpt = leaderEpochCache.get.epochForOffset(curLocalLogStartOffset) + if (epochOpt.isPresent) Optional.of(epochOpt.getAsInt) else Optional.empty() + } else { + Optional.empty() + } - Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curLocalLogStartOffset, epochOpt)) + Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curLocalLogStartOffset, epochResult)) } else if (targetTimestamp == ListOffsetsRequest.LATEST_TIMESTAMP) { Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, logEndOffset, latestEpochAsOptional(leaderEpochCache))) + } else if (targetTimestamp == ListOffsetsRequest.LATEST_TIERED_TIMESTAMP) { + if (remoteLogEnabled()) { + val curHighestRemoteOffset = highestOffsetInRemoteStorage() + + val epochResult: Optional[Integer] = + if (leaderEpochCache.isDefined) { + val epochOpt = leaderEpochCache.get.epochForOffset(curHighestRemoteOffset) + if (epochOpt.isPresent) { + Optional.of(epochOpt.getAsInt) + } else if (curHighestRemoteOffset == -1) { + Optional.of(RecordBatch.NO_PARTITION_LEADER_EPOCH) + } else { + Optional.empty() + } + } else { + Optional.empty() + } + + Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, curHighestRemoteOffset, epochResult)) + } else { + Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, -1L, Optional.of(-1))) + } } else if (targetTimestamp == ListOffsetsRequest.MAX_TIMESTAMP) { // Cache to avoid race conditions. `toBuffer` is faster than most alternatives and provides // constant time access while being safe to use with concurrent collections unlike `toArray`. @@ -1454,7 +1476,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, // 1. they are uploaded to the remote storage // 2. log-start-offset was incremented higher than the largest offset in the candidate segment if (remoteLogEnabled()) { - (upperBoundOffset > 0 && upperBoundOffset - 1 <= highestOffsetInRemoteStorage) || + (upperBoundOffset > 0 && upperBoundOffset - 1 <= highestOffsetInRemoteStorage()) || allowDeletionDueToLogStartOffsetIncremented } else { true @@ -1588,13 +1610,13 @@ class UnifiedLog(@volatile var logStartOffset: Long, * The log size in bytes for all segments that are only in local log but not yet in remote log. */ def onlyLocalLogSegmentsSize: Long = - UnifiedLog.sizeInBytes(logSegments.stream.filter(_.baseOffset >= highestOffsetInRemoteStorage).collect(Collectors.toList[LogSegment])) + UnifiedLog.sizeInBytes(logSegments.stream.filter(_.baseOffset >= highestOffsetInRemoteStorage()).collect(Collectors.toList[LogSegment])) /** * The number of segments that are only in local log but not yet in remote log. */ def onlyLocalLogSegmentsCount: Long = - logSegments.stream().filter(_.baseOffset >= highestOffsetInRemoteStorage).count() + logSegments.stream().filter(_.baseOffset >= highestOffsetInRemoteStorage()).count() /** * The offset of the next message that will be appended to the log diff --git a/core/src/main/scala/kafka/log/streamaspect/ElasticLog.scala b/core/src/main/scala/kafka/log/streamaspect/ElasticLog.scala index ba4e15f39f..914f1a1a28 100644 --- a/core/src/main/scala/kafka/log/streamaspect/ElasticLog.scala +++ b/core/src/main/scala/kafka/log/streamaspect/ElasticLog.scala @@ -172,7 +172,7 @@ class ElasticLog(val metaStream: MetaStream, } } - override private[log] def append(lastOffset: Long, largestTimestamp: Long, shallowOffsetOfMaxTimestamp: Long, + override private[log] def append(lastOffset: Long, largestTimestamp: Long, offsetOfMaxTimestamp: Long, records: MemoryRecords): Unit = { val activeSegment = segments.activeSegment val startTimestamp = time.nanoseconds() @@ -185,7 +185,7 @@ class ElasticLog(val metaStream: MetaStream, APPEND_PERMIT_ACQUIRE_FAIL_TIME_HIST.update(System.nanoTime() - startTimestamp) } - activeSegment.append(lastOffset, largestTimestamp, shallowOffsetOfMaxTimestamp, records) + activeSegment.append(lastOffset, largestTimestamp, offsetOfMaxTimestamp, records) APPEND_TIME_HIST.update(System.nanoTime() - startTimestamp) val endOffset = lastOffset + 1 diff --git a/core/src/main/scala/kafka/log/streamaspect/ElasticLogSegment.java b/core/src/main/scala/kafka/log/streamaspect/ElasticLogSegment.java index 18ea8d70c4..a71ac3bdf5 100644 --- a/core/src/main/scala/kafka/log/streamaspect/ElasticLogSegment.java +++ b/core/src/main/scala/kafka/log/streamaspect/ElasticLogSegment.java @@ -189,12 +189,12 @@ private boolean canConvertToRelativeOffset(long offset) { public void append( long largestOffset, long largestTimestampMs, - long shallowOffsetOfMaxTimestamp, + long offsetOfMaxTimestamp, MemoryRecords records) throws IOException { if (records.sizeInBytes() > 0) { if (LOGGER.isTraceEnabled()) { - LOGGER.trace("Inserting {} bytes at end offset {} at position {} with largest timestamp {} at shallow offset {}", - records.sizeInBytes(), largestOffset, log.sizeInBytes(), largestTimestampMs, shallowOffsetOfMaxTimestamp); + LOGGER.trace("Inserting {} bytes at end offset {} at position {} with largest timestamp {} at offset {}", + records.sizeInBytes(), largestOffset, log.sizeInBytes(), largestTimestampMs, offsetOfMaxTimestamp); } int physicalPosition = log.sizeInBytes(); if (physicalPosition == 0) { @@ -207,7 +207,7 @@ public void append( LOGGER.trace("Appended {} to {} at end offset {}", appendedBytes, log, largestOffset); // Update the in memory max timestamp and corresponding offset. if (largestTimestampMs > maxTimestampSoFar()) { - maxTimestampAndOffsetSoFar = new TimestampOffset(largestTimestampMs, shallowOffsetOfMaxTimestamp); + maxTimestampAndOffsetSoFar = new TimestampOffset(largestTimestampMs, offsetOfMaxTimestamp); } // append an entry to the index (if needed) if (bytesSinceLastIndexEntry > indexIntervalBytes) { diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index e41b650066..514b08797a 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -17,10 +17,15 @@ package kafka.raft import kafka.log.UnifiedLog +import kafka.raft.KafkaMetadataLog.FullTruncation +import kafka.raft.KafkaMetadataLog.RetentionMsBreach +import kafka.raft.KafkaMetadataLog.RetentionSizeBreach +import kafka.raft.KafkaMetadataLog.SnapshotDeletionReason +import kafka.raft.KafkaMetadataLog.UnknownReason import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, MetadataLogSegmentMinBytesProp} -import kafka.server.{BrokerTopicStats, KafkaConfig, RequestLocal} +import kafka.server.{BrokerTopicStats, RequestLocal} import kafka.utils.{CoreUtils, Logging} -import org.apache.kafka.common.config.{AbstractConfig, TopicConfig} +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.InvalidConfigurationException import org.apache.kafka.common.record.{MemoryRecords, Records} import org.apache.kafka.common.utils.Time @@ -179,7 +184,7 @@ final class KafkaMetadataLog private ( (false, mutable.TreeMap.empty[OffsetAndEpoch, Option[FileRawSnapshotReader]]) } - removeSnapshots(forgottenSnapshots) + removeSnapshots(forgottenSnapshots, FullTruncation) truncated } @@ -340,6 +345,10 @@ final class KafkaMetadataLog private ( * This method is thread-safe */ override def deleteBeforeSnapshot(snapshotId: OffsetAndEpoch): Boolean = { + deleteBeforeSnapshot(snapshotId, UnknownReason) + } + + private def deleteBeforeSnapshot(snapshotId: OffsetAndEpoch, reason: SnapshotDeletionReason): Boolean = { val (deleted, forgottenSnapshots) = snapshots synchronized { latestSnapshotId().asScala match { case Some(latestSnapshotId) if @@ -348,14 +357,15 @@ final class KafkaMetadataLog private ( snapshotId.offset <= latestSnapshotId.offset && log.maybeIncrementLogStartOffset(snapshotId.offset, LogStartOffsetIncrementReason.SnapshotGenerated) => // Delete all segments that have a "last offset" less than the log start offset - log.deleteOldSegments() + val deletedSegments = log.deleteOldSegments() // Remove older snapshots from the snapshots cache - (true, forgetSnapshotsBefore(snapshotId)) + val forgottenSnapshots = forgetSnapshotsBefore(snapshotId) + (deletedSegments != 0 || forgottenSnapshots.nonEmpty, forgottenSnapshots) case _ => - (false, mutable.TreeMap.empty[OffsetAndEpoch, Option[FileRawSnapshotReader]]) + (false, mutable.TreeMap.empty[OffsetAndEpoch, Option[FileRawSnapshotReader]]) } } - removeSnapshots(forgottenSnapshots) + removeSnapshots(forgottenSnapshots, reason) deleted } @@ -404,21 +414,33 @@ final class KafkaMetadataLog private ( * all cases. * * For the given predicate, we are testing if the snapshot identified by the first argument should be deleted. + * The predicate returns a Some with the reason if the snapshot should be deleted and a None if the snapshot + * should not be deleted */ - private def cleanSnapshots(predicate: OffsetAndEpoch => Boolean): Boolean = { - if (snapshots.size < 2) + private def cleanSnapshots(predicate: OffsetAndEpoch => Option[SnapshotDeletionReason]): Boolean = { + if (snapshots.size < 2) { return false + } var didClean = false snapshots.keys.toSeq.sliding(2).foreach { case Seq(snapshot: OffsetAndEpoch, nextSnapshot: OffsetAndEpoch) => - if (predicate(snapshot) && deleteBeforeSnapshot(nextSnapshot)) { - didClean = true - } else { - return didClean + predicate(snapshot) match { + case Some(reason) => + if (deleteBeforeSnapshot(nextSnapshot, reason)) { + didClean = true + } else { + return didClean + } + + case None => + return didClean + } + case _ => false // Shouldn't get here with the sliding window } + didClean } @@ -427,13 +449,13 @@ final class KafkaMetadataLog private ( return false // Keep deleting snapshots as long as the - def shouldClean(snapshotId: OffsetAndEpoch): Boolean = { - val now = time.milliseconds() - readSnapshotTimestamp(snapshotId).exists { timestamp => + def shouldClean(snapshotId: OffsetAndEpoch): Option[SnapshotDeletionReason] = { + readSnapshotTimestamp(snapshotId).flatMap { timestamp => + val now = time.milliseconds() if (now - timestamp > config.retentionMillis) { - true + Some(RetentionMsBreach(now, timestamp, config.retentionMillis)) } else { - false + None } } } @@ -450,13 +472,14 @@ final class KafkaMetadataLog private ( var snapshotTotalSize: Long = snapshotSizes.values.sum // Keep deleting snapshots and segments as long as we exceed the retention size - def shouldClean(snapshotId: OffsetAndEpoch): Boolean = { - snapshotSizes.get(snapshotId).exists { snapshotSize => + def shouldClean(snapshotId: OffsetAndEpoch): Option[SnapshotDeletionReason] = { + snapshotSizes.get(snapshotId).flatMap { snapshotSize => if (log.size + snapshotTotalSize > config.retentionMaxBytes) { + val oldSnapshotTotalSize = snapshotTotalSize snapshotTotalSize -= snapshotSize - true + Some(RetentionSizeBreach(log.size, oldSnapshotTotalSize, config.retentionMaxBytes)) } else { - false + None } } } @@ -485,10 +508,11 @@ final class KafkaMetadataLog private ( * given snapshots after some delay. */ private def removeSnapshots( - expiredSnapshots: mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]] + expiredSnapshots: mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]], + reason: SnapshotDeletionReason, ): Unit = { expiredSnapshots.foreach { case (snapshotId, _) => - info(s"Marking snapshot $snapshotId for deletion") + info(reason.reason(snapshotId)) Snapshots.markForDelete(log.dir.toPath, snapshotId) } @@ -516,32 +540,6 @@ final class KafkaMetadataLog private ( } } -object MetadataLogConfig { - def apply(config: AbstractConfig, maxBatchSizeInBytes: Int, maxFetchSizeInBytes: Int): MetadataLogConfig = { - new MetadataLogConfig( - config.getInt(KafkaConfig.MetadataLogSegmentBytesProp), - config.getInt(KafkaConfig.MetadataLogSegmentMinBytesProp), - config.getLong(KafkaConfig.MetadataLogSegmentMillisProp), - config.getLong(KafkaConfig.MetadataMaxRetentionBytesProp), - config.getLong(KafkaConfig.MetadataMaxRetentionMillisProp), - maxBatchSizeInBytes, - maxFetchSizeInBytes, - LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, - config.getInt(KafkaConfig.NodeIdProp) - ) - } -} - -case class MetadataLogConfig(logSegmentBytes: Int, - logSegmentMinBytes: Int, - logSegmentMillis: Long, - retentionMaxBytes: Long, - retentionMillis: Long, - maxBatchSizeInBytes: Int, - maxFetchSizeInBytes: Int, - fileDeleteDelayMs: Long, - nodeId: Int) - object KafkaMetadataLog extends Logging { def apply( topicPartition: TopicPartition, @@ -677,4 +675,38 @@ object KafkaMetadataLog extends Logging { Snapshots.deleteIfExists(logDir, snapshotId) } } + + private sealed trait SnapshotDeletionReason { + def reason(snapshotId: OffsetAndEpoch): String + } + + private final case class RetentionMsBreach(now: Long, timestamp: Long, retentionMillis: Long) extends SnapshotDeletionReason { + override def reason(snapshotId: OffsetAndEpoch): String = { + s"""Marking snapshot $snapshotId for deletion because its timestamp ($timestamp) is now ($now) older than the + |retention ($retentionMillis)""".stripMargin + } + } + + private final case class RetentionSizeBreach( + logSize: Long, + snapshotsSize: Long, + retentionMaxBytes: Long + ) extends SnapshotDeletionReason { + override def reason(snapshotId: OffsetAndEpoch): String = { + s"""Marking snapshot $snapshotId for deletion because the log size ($logSize) and snapshots size ($snapshotsSize) + |is greater than $retentionMaxBytes""".stripMargin + } + } + + private final object FullTruncation extends SnapshotDeletionReason { + override def reason(snapshotId: OffsetAndEpoch): String = { + s"Marking snapshot $snapshotId for deletion because the partition was fully truncated" + } + } + + private final object UnknownReason extends SnapshotDeletionReason { + override def reason(snapshotId: OffsetAndEpoch): String = { + s"Marking snapshot $snapshotId for deletion for unknown reason" + } + } } diff --git a/core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala b/core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala index 59f75d4b11..ec909580a8 100644 --- a/core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala +++ b/core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala @@ -32,7 +32,7 @@ import org.apache.kafka.server.util.{InterBrokerSendThread, RequestAndCompletion import java.util import java.util.concurrent.TimeUnit -import scala.collection.{Set, Seq, mutable} +import scala.collection.{Seq, mutable} import scala.jdk.CollectionConverters._ object AddPartitionsToTxnManager { @@ -42,6 +42,17 @@ object AddPartitionsToTxnManager { val VerificationTimeMsMetricName = "VerificationTimeMs" } +/** + * This is an enum which handles the Partition Response based on the Request Version and the exact operation + * defaultError: This is the default workflow which maps to cases when the Produce Request Version or the Txn_offset_commit request was lower than the first version supporting the new Error Class + * genericError: This maps to the case when the clients are updated to handle the AbortableTxnException + * addPartition: This is a WIP. To be updated as a part of KIP-890 Part 2 + */ +sealed trait SupportedOperation +case object defaultError extends SupportedOperation +case object genericError extends SupportedOperation +case object addPartition extends SupportedOperation + /* * Data structure to hold the transactional data to send to a node. Note -- at most one request per transactional ID * will exist at a time in the map. If a given transactional ID exists in the map, and a new request with the same ID @@ -49,7 +60,8 @@ object AddPartitionsToTxnManager { */ class TransactionDataAndCallbacks(val transactionData: AddPartitionsToTxnTransactionCollection, val callbacks: mutable.Map[String, AddPartitionsToTxnManager.AppendCallback], - val startTimeMs: mutable.Map[String, Long]) + val startTimeMs: mutable.Map[String, Long], + val supportedOperation: SupportedOperation) class AddPartitionsToTxnManager( config: KafkaConfig, @@ -66,6 +78,7 @@ class AddPartitionsToTxnManager( this.logIdent = logPrefix + private val interBrokerListenerName = config.interBrokerListenerName private val inflightNodes = mutable.HashSet[Node]() private val nodesToTransactions = mutable.Map[Node, TransactionDataAndCallbacks]() @@ -78,12 +91,12 @@ class AddPartitionsToTxnManager( producerId: Long, producerEpoch: Short, topicPartitions: Seq[TopicPartition], - callback: AddPartitionsToTxnManager.AppendCallback + callback: AddPartitionsToTxnManager.AppendCallback, + supportedOperation: SupportedOperation ): Unit = { - val (error, node) = getTransactionCoordinator(partitionFor(transactionalId)) - - if (error != Errors.NONE) { - callback(topicPartitions.map(tp => tp -> error).toMap) + val coordinatorNode = getTransactionCoordinator(partitionFor(transactionalId)) + if (coordinatorNode.isEmpty) { + callback(topicPartitions.map(tp => tp -> Errors.COORDINATOR_NOT_AVAILABLE).toMap) } else { val topicCollection = new AddPartitionsToTxnTopicCollection() topicPartitions.groupBy(_.topic).forKeyValue { (topic, tps) => @@ -99,14 +112,16 @@ class AddPartitionsToTxnManager( .setVerifyOnly(true) .setTopics(topicCollection) - addTxnData(node, transactionData, callback) + addTxnData(coordinatorNode.get, transactionData, callback, supportedOperation) + } } private def addTxnData( node: Node, transactionData: AddPartitionsToTxnTransaction, - callback: AddPartitionsToTxnManager.AppendCallback + callback: AddPartitionsToTxnManager.AppendCallback, + supportedOperation: SupportedOperation ): Unit = { nodesToTransactions.synchronized { val curTime = time.milliseconds() @@ -115,7 +130,8 @@ class AddPartitionsToTxnManager( new TransactionDataAndCallbacks( new AddPartitionsToTxnTransactionCollection(1), mutable.Map[String, AddPartitionsToTxnManager.AppendCallback](), - mutable.Map[String, Long]())) + mutable.Map[String, Long](), + supportedOperation)) val existingTransactionData = existingNodeAndTransactionData.transactionData.find(transactionData.transactionalId) @@ -146,31 +162,10 @@ class AddPartitionsToTxnManager( } } - private def getTransactionCoordinator(partition: Int): (Errors, Node) = { - val listenerName = config.interBrokerListenerName - - val topicMetadata = metadataCache.getTopicMetadata(Set(Topic.TRANSACTION_STATE_TOPIC_NAME), listenerName) - - if (topicMetadata.headOption.isEmpty) { - // If topic is not created, then the transaction is definitely not started. - (Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode) - } else { - if (topicMetadata.head.errorCode != Errors.NONE.code) { - (Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode) - } else { - val coordinatorEndpoint = topicMetadata.head.partitions.asScala - .find(_.partitionIndex == partition) - .filter(_.leaderId != MetadataResponse.NO_LEADER_ID) - .flatMap(metadata => metadataCache.getAliveBrokerNode(metadata.leaderId, listenerName)) - - coordinatorEndpoint match { - case Some(endpoint) => - (Errors.NONE, endpoint) - case _ => - (Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode) - } - } - } + private def getTransactionCoordinator(partition: Int): Option[Node] = { + metadataCache.getPartitionInfo(Topic.TRANSACTION_STATE_TOPIC_NAME, partition) + .filter(_.leader != MetadataResponse.NO_LEADER_ID) + .flatMap(metadata => metadataCache.getAliveBrokerNode(metadata.leader, interBrokerListenerName)) } private def topicPartitionsToError(transactionData: AddPartitionsToTxnTransaction, error: Errors): Map[TopicPartition, Errors] = { @@ -231,6 +226,8 @@ class AddPartitionsToTxnManager( val code = if (partitionResult.partitionErrorCode == Errors.PRODUCER_FENCED.code) Errors.INVALID_PRODUCER_EPOCH.code + else if (partitionResult.partitionErrorCode() == Errors.ABORTABLE_TRANSACTION.code && transactionDataAndCallbacks.supportedOperation != genericError) // For backward compatibility with clients. + Errors.INVALID_TXN_STATE.code else partitionResult.partitionErrorCode unverified.put(tp, Errors.forCode(code)) diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala index 10c4ba05ce..1a10c6f659 100644 --- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala +++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala @@ -23,7 +23,7 @@ import kafka.utils.Logging import org.apache.kafka.clients.ClientResponse import org.apache.kafka.common.Uuid import org.apache.kafka.common.message.BrokerRegistrationRequestData.ListenerCollection -import org.apache.kafka.common.message.{BrokerHeartbeatRequestData, BrokerHeartbeatResponseData, BrokerRegistrationRequestData, BrokerRegistrationResponseData} +import org.apache.kafka.common.message.{BrokerHeartbeatRequestData, BrokerRegistrationRequestData} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse} import org.apache.kafka.metadata.{BrokerState, VersionRange} @@ -166,6 +166,19 @@ class BrokerLifecycleManager( */ private var registered = false + /** + * True if a request has been sent and a response or timeout has not yet been processed. + * This variable can only be read or written from the event queue thread. + */ + private var communicationInFlight = false + + /** + * True if we should schedule the next communication immediately. This is used to delay + * an immediate scheduling of a communication event if one is already in flight. + * This variable can only be read or written from the event queue thread. + */ + private var nextSchedulingShouldBeImmediate = false + /** * True if the initial registration succeeded. This variable can only be read or * written from the event queue thread. @@ -377,10 +390,30 @@ class BrokerLifecycleManager( } _channelManager.sendRequest(new BrokerRegistrationRequest.Builder(data), new BrokerRegistrationResponseHandler()) + communicationInFlight = true } + // the response handler is not invoked from the event handler thread, + // so it is not safe to update state here, instead, schedule an event + // to continue handling the response on the event handler thread private class BrokerRegistrationResponseHandler extends ControllerRequestCompletionHandler { override def onComplete(response: ClientResponse): Unit = { + eventQueue.prepend(new BrokerRegistrationResponseEvent(response, false)) + } + + override def onTimeout(): Unit = { + info(s"Unable to register the broker because the RPC got timed out before it could be sent.") + eventQueue.prepend(new BrokerRegistrationResponseEvent(null, true)) + } + } + + private class BrokerRegistrationResponseEvent(response: ClientResponse, timedOut: Boolean) extends EventQueue.Event { + override def run(): Unit = { + communicationInFlight = false + if (timedOut) { + scheduleNextCommunicationAfterFailure() + return + } if (response.authenticationException() != null) { error(s"Unable to register broker $nodeId because of an authentication exception.", response.authenticationException()) @@ -400,10 +433,12 @@ class BrokerLifecycleManager( val message = response.responseBody().asInstanceOf[BrokerRegistrationResponse] val errorCode = Errors.forCode(message.data().errorCode()) if (errorCode == Errors.NONE) { - // this response handler is not invoked from the event handler thread, - // and processing a successful registration response requires updating - // state, so to continue we need to schedule an event - eventQueue.prepend(new BrokerRegistrationResponseEvent(message.data())) + failedAttempts = 0 + _brokerEpoch = message.data().brokerEpoch() + registered = true + initialRegistrationSucceeded = true + info(s"Successfully registered broker $nodeId with broker epoch ${_brokerEpoch}") + scheduleNextCommunicationImmediately() // Immediately send a heartbeat } else { info(s"Unable to register broker $nodeId because the controller returned " + s"error $errorCode") @@ -411,22 +446,6 @@ class BrokerLifecycleManager( } } } - - override def onTimeout(): Unit = { - info(s"Unable to register the broker because the RPC got timed out before it could be sent.") - scheduleNextCommunicationAfterFailure() - } - } - - private class BrokerRegistrationResponseEvent(response: BrokerRegistrationResponseData) extends EventQueue.Event { - override def run(): Unit = { - failedAttempts = 0 - _brokerEpoch = response.brokerEpoch() - registered = true - initialRegistrationSucceeded = true - info(s"Successfully registered broker $nodeId with broker epoch ${_brokerEpoch}") - scheduleNextCommunicationImmediately() // Immediately send a heartbeat - } } private def sendBrokerHeartbeat(): Unit = { @@ -443,10 +462,30 @@ class BrokerLifecycleManager( } val handler = new BrokerHeartbeatResponseHandler() _channelManager.sendRequest(new BrokerHeartbeatRequest.Builder(data), handler) + communicationInFlight = true } + // the response handler is not invoked from the event handler thread, + // so it is not safe to update state here, instead, schedule an event + // to continue handling the response on the event handler thread private class BrokerHeartbeatResponseHandler extends ControllerRequestCompletionHandler { override def onComplete(response: ClientResponse): Unit = { + eventQueue.prepend(new BrokerHeartbeatResponseEvent(response, false)) + } + + override def onTimeout(): Unit = { + info("Unable to send a heartbeat because the RPC got timed out before it could be sent.") + eventQueue.prepend(new BrokerHeartbeatResponseEvent(null, true)) + } + } + + private class BrokerHeartbeatResponseEvent(response: ClientResponse, timedOut: Boolean) extends EventQueue.Event { + override def run(): Unit = { + communicationInFlight = false + if (timedOut) { + scheduleNextCommunicationAfterFailure() + return + } if (response.authenticationException() != null) { error(s"Unable to send broker heartbeat for $nodeId because of an " + "authentication exception.", response.authenticationException()) @@ -466,82 +505,72 @@ class BrokerLifecycleManager( val message = response.responseBody().asInstanceOf[BrokerHeartbeatResponse] val errorCode = Errors.forCode(message.data().errorCode()) if (errorCode == Errors.NONE) { - // this response handler is not invoked from the event handler thread, - // and processing a successful heartbeat response requires updating - // state, so to continue we need to schedule an event - eventQueue.prepend(new BrokerHeartbeatResponseEvent(message.data())) + val responseData = message.data() + failedAttempts = 0 + _state match { + case BrokerState.STARTING => + if (responseData.isCaughtUp) { + info(s"The broker has caught up. Transitioning from STARTING to RECOVERY.") + _state = BrokerState.RECOVERY + initialCatchUpFuture.complete(null) + } else { + debug(s"The broker is STARTING. Still waiting to catch up with cluster metadata.") + } + // Schedule the heartbeat after only 10 ms so that in the case where + // there is no recovery work to be done, we start up a bit quicker. + scheduleNextCommunication(NANOSECONDS.convert(10, MILLISECONDS)) + case BrokerState.RECOVERY => + if (!responseData.isFenced) { + info(s"The broker has been unfenced. Transitioning from RECOVERY to RUNNING.") + initialUnfenceFuture.complete(null) + _state = BrokerState.RUNNING + } else { + info(s"The broker is in RECOVERY.") + } + scheduleNextCommunicationAfterSuccess() + case BrokerState.RUNNING => + debug(s"The broker is RUNNING. Processing heartbeat response.") + scheduleNextCommunicationAfterSuccess() + case BrokerState.PENDING_CONTROLLED_SHUTDOWN => + if (!responseData.shouldShutDown()) { + info(s"The broker is in PENDING_CONTROLLED_SHUTDOWN state, still waiting " + + "for the active controller.") + if (!gotControlledShutdownResponse) { + // If this is the first pending controlled shutdown response we got, + // schedule our next heartbeat a little bit sooner than we usually would. + // In the case where controlled shutdown completes quickly, this will + // speed things up a little bit. + scheduleNextCommunication(NANOSECONDS.convert(50, MILLISECONDS)) + } else { + scheduleNextCommunicationAfterSuccess() + } + } else { + info(s"The controller has asked us to exit controlled shutdown.") + beginShutdown() + } + gotControlledShutdownResponse = true + case BrokerState.SHUTTING_DOWN => + info(s"The broker is SHUTTING_DOWN. Ignoring heartbeat response.") + case _ => + error(s"Unexpected broker state ${_state}") + scheduleNextCommunicationAfterSuccess() + } } else { warn(s"Broker $nodeId sent a heartbeat request but received error $errorCode.") scheduleNextCommunicationAfterFailure() } } } - - override def onTimeout(): Unit = { - info("Unable to send a heartbeat because the RPC got timed out before it could be sent.") - scheduleNextCommunicationAfterFailure() - } } - private class BrokerHeartbeatResponseEvent(response: BrokerHeartbeatResponseData) extends EventQueue.Event { - override def run(): Unit = { - failedAttempts = 0 - _state match { - case BrokerState.STARTING => - if (response.isCaughtUp) { - info(s"The broker has caught up. Transitioning from STARTING to RECOVERY.") - _state = BrokerState.RECOVERY - initialCatchUpFuture.complete(null) - } else { - debug(s"The broker is STARTING. Still waiting to catch up with cluster metadata.") - } - // Schedule the heartbeat after only 10 ms so that in the case where - // there is no recovery work to be done, we start up a bit quicker. - scheduleNextCommunication(NANOSECONDS.convert(10, MILLISECONDS)) - case BrokerState.RECOVERY => - if (!response.isFenced) { - info(s"The broker has been unfenced. Transitioning from RECOVERY to RUNNING.") - initialUnfenceFuture.complete(null) - _state = BrokerState.RUNNING - } else { - info(s"The broker is in RECOVERY.") - } - scheduleNextCommunicationAfterSuccess() - case BrokerState.RUNNING => - debug(s"The broker is RUNNING. Processing heartbeat response.") - scheduleNextCommunicationAfterSuccess() - case BrokerState.PENDING_CONTROLLED_SHUTDOWN => - if (!response.shouldShutDown()) { - info(s"The broker is in PENDING_CONTROLLED_SHUTDOWN state, still waiting " + - "for the active controller.") - if (!gotControlledShutdownResponse) { - // If this is the first pending controlled shutdown response we got, - // schedule our next heartbeat a little bit sooner than we usually would. - // In the case where controlled shutdown completes quickly, this will - // speed things up a little bit. - scheduleNextCommunication(NANOSECONDS.convert(50, MILLISECONDS)) - } else { - scheduleNextCommunicationAfterSuccess() - } - } else { - info(s"The controller has asked us to exit controlled shutdown.") - beginShutdown() - } - gotControlledShutdownResponse = true - case BrokerState.SHUTTING_DOWN => - info(s"The broker is SHUTTING_DOWN. Ignoring heartbeat response.") - case _ => - error(s"Unexpected broker state ${_state}") - scheduleNextCommunicationAfterSuccess() - } - } + private def scheduleNextCommunicationImmediately(): Unit = { + scheduleNextCommunication(0) } - private def scheduleNextCommunicationImmediately(): Unit = scheduleNextCommunication(0) - private def scheduleNextCommunicationAfterFailure(): Unit = { val delayMs = resendExponentialBackoff.backoff(failedAttempts) failedAttempts = failedAttempts + 1 + nextSchedulingShouldBeImmediate = false // never immediately reschedule after a failure scheduleNextCommunication(NANOSECONDS.convert(delayMs, MILLISECONDS)) } @@ -551,9 +580,11 @@ class BrokerLifecycleManager( } private def scheduleNextCommunication(intervalNs: Long): Unit = { - trace(s"Scheduling next communication at ${MILLISECONDS.convert(intervalNs, NANOSECONDS)} " + + val adjustedIntervalNs = if (nextSchedulingShouldBeImmediate) 0 else intervalNs + nextSchedulingShouldBeImmediate = false + trace(s"Scheduling next communication at ${MILLISECONDS.convert(adjustedIntervalNs, NANOSECONDS)} " + "ms from now.") - val deadlineNs = time.nanoseconds() + intervalNs + val deadlineNs = time.nanoseconds() + adjustedIntervalNs eventQueue.scheduleDeferred("communication", new DeadlineFunction(deadlineNs), new CommunicationEvent()) @@ -570,7 +601,10 @@ class BrokerLifecycleManager( private class CommunicationEvent extends EventQueue.Event { override def run(): Unit = { - if (registered) { + if (communicationInFlight) { + trace("Delaying communication because there is already one in flight.") + nextSchedulingShouldBeImmediate = true + } else if (registered) { sendBrokerHeartbeat() } else { sendBrokerRegistration() diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 59950d9f1a..06ddd58ebe 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -99,7 +99,6 @@ class BrokerServer( var status: ProcessStatus = SHUTDOWN @volatile var dataPlaneRequestProcessor: KafkaApis = _ - var controlPlaneRequestProcessor: KafkaApis = _ var authorizer: Option[Authorizer] = None @volatile var socketServer: SocketServer = _ @@ -674,8 +673,6 @@ class BrokerServer( CoreUtils.swallow(dataPlaneRequestHandlerPool.shutdown(), this) if (dataPlaneRequestProcessor != null) CoreUtils.swallow(dataPlaneRequestProcessor.close(), this) - if (controlPlaneRequestProcessor != null) - CoreUtils.swallow(controlPlaneRequestProcessor.close(), this) CoreUtils.swallow(authorizer.foreach(_.close()), this) // AutoMQ for Kafka inject start diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 2d2043f8dd..b02d2f4430 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -721,7 +721,7 @@ class KafkaApis(val requestChannel: RequestChannel, sendResponseCallback(Map.empty) else { val internalTopicsAllowed = request.header.clientId == AdminUtils.ADMIN_CLIENT_ID - + val supportedOperation = if (request.header.apiVersion > 10) genericError else defaultError // call the replica manager to append messages to the replicas replicaManager.handleProduceAppend( timeout = produceRequest.timeout.toLong, @@ -731,7 +731,8 @@ class KafkaApis(val requestChannel: RequestChannel, entriesPerPartition = authorizedRequestInfo, responseCallback = sendResponseCallback, recordValidationStatsCallback = processingStatsCallback, - requestLocal = requestLocal) + requestLocal = requestLocal, + supportedOperation = supportedOperation) // if the request is put into the purgatory, it will have a held reference and hence cannot be garbage collected; // hence we clear its data here in order to let GC reclaim its memory since it is already appended to log diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 701a19928c..56952da622 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -53,7 +53,7 @@ import org.apache.kafka.server.config.{Defaults, ServerTopicConfigSynonyms} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.util.Csv -import org.apache.kafka.storage.internals.log.{LogConfig, ProducerStateManagerConfig} +import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.LogConfig.MessageFormatVersion import org.apache.zookeeper.client.ZKClientConfig @@ -215,17 +215,6 @@ object KafkaConfig { val LogRetentionBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.RETENTION_BYTES_CONFIG) val LogCleanupIntervalMsProp = LogConfigPrefix + "retention.check.interval.ms" val LogCleanupPolicyProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.CLEANUP_POLICY_CONFIG) - val LogCleanerThreadsProp = LogConfigPrefix + "cleaner.threads" - val LogCleanerIoMaxBytesPerSecondProp = LogConfigPrefix + "cleaner.io.max.bytes.per.second" - val LogCleanerDedupeBufferSizeProp = LogConfigPrefix + "cleaner.dedupe.buffer.size" - val LogCleanerIoBufferSizeProp = LogConfigPrefix + "cleaner.io.buffer.size" - val LogCleanerDedupeBufferLoadFactorProp = LogConfigPrefix + "cleaner.io.buffer.load.factor" - val LogCleanerBackoffMsProp = LogConfigPrefix + "cleaner.backoff.ms" - val LogCleanerMinCleanRatioProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG) - val LogCleanerEnableProp = LogConfigPrefix + "cleaner.enable" - val LogCleanerDeleteRetentionMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.DELETE_RETENTION_MS_CONFIG) - val LogCleanerMinCompactionLagMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG) - val LogCleanerMaxCompactionLagMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG) val LogIndexSizeMaxBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG) val LogIndexIntervalBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG) val LogFlushIntervalMessagesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG) @@ -768,25 +757,6 @@ object KafkaConfig { val LogRetentionBytesDoc = "The maximum size of the log before deleting it" val LogCleanupIntervalMsDoc = "The frequency in milliseconds that the log cleaner checks whether any log is eligible for deletion" val LogCleanupPolicyDoc = "The default cleanup policy for segments beyond the retention window. A comma separated list of valid policies. Valid policies are: \"delete\" and \"compact\"" - val LogCleanerThreadsDoc = "The number of background threads to use for log cleaning" - val LogCleanerIoMaxBytesPerSecondDoc = "The log cleaner will be throttled so that the sum of its read and write i/o will be less than this value on average" - val LogCleanerDedupeBufferSizeDoc = "The total memory used for log deduplication across all cleaner threads" - val LogCleanerIoBufferSizeDoc = "The total memory used for log cleaner I/O buffers across all cleaner threads" - val LogCleanerDedupeBufferLoadFactorDoc = "Log cleaner dedupe buffer load factor. The percentage full the dedupe buffer can become. A higher value " + - "will allow more log to be cleaned at once but will lead to more hash collisions" - val LogCleanerBackoffMsDoc = "The amount of time to sleep when there are no logs to clean" - val LogCleanerMinCleanRatioDoc = "The minimum ratio of dirty log to total log for a log to eligible for cleaning. " + - "If the " + LogCleanerMaxCompactionLagMsProp + " or the " + LogCleanerMinCompactionLagMsProp + - " configurations are also specified, then the log compactor considers the log eligible for compaction " + - "as soon as either: (i) the dirty ratio threshold has been met and the log has had dirty (uncompacted) " + - "records for at least the " + LogCleanerMinCompactionLagMsProp + " duration, or (ii) if the log has had " + - "dirty (uncompacted) records for at most the " + LogCleanerMaxCompactionLagMsProp + " period." - val LogCleanerEnableDoc = "Enable the log cleaner process to run on the server. Should be enabled if using any topics with a cleanup.policy=compact including the internal offsets topic. If disabled those topics will not be compacted and continually grow in size." - val LogCleanerDeleteRetentionMsDoc = "The amount of time to retain tombstone message markers for log compacted topics. This setting also gives a bound " + - "on the time in which a consumer must complete a read if they begin from offset 0 to ensure that they get a valid snapshot of the final stage (otherwise " + - "tombstones messages may be collected before a consumer completes their scan)." - val LogCleanerMinCompactionLagMsDoc = "The minimum time a message will remain uncompacted in the log. Only applicable for logs that are being compacted." - val LogCleanerMaxCompactionLagMsDoc = "The maximum time a message will remain ineligible for compaction in the log. Only applicable for logs that are being compacted." val LogIndexSizeMaxBytesDoc = "The maximum size in bytes of the offset index" val LogIndexIntervalBytesDoc = "The interval with which we add an entry to the offset index." val LogFlushIntervalMessagesDoc = "The number of messages accumulated on a log partition before messages are flushed to disk." @@ -1191,17 +1161,17 @@ object KafkaConfig { .define(LogRetentionBytesProp, LONG, LogConfig.DEFAULT_RETENTION_BYTES, HIGH, LogRetentionBytesDoc) .define(LogCleanupIntervalMsProp, LONG, Defaults.LOG_CLEANUP_INTERVAL_MS, atLeast(1), MEDIUM, LogCleanupIntervalMsDoc) .define(LogCleanupPolicyProp, LIST, LogConfig.DEFAULT_CLEANUP_POLICY, ValidList.in(TopicConfig.CLEANUP_POLICY_COMPACT, TopicConfig.CLEANUP_POLICY_DELETE), MEDIUM, LogCleanupPolicyDoc) - .define(LogCleanerThreadsProp, INT, Defaults.LOG_CLEANER_THREADS, atLeast(0), MEDIUM, LogCleanerThreadsDoc) - .define(LogCleanerIoMaxBytesPerSecondProp, DOUBLE, Defaults.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND, MEDIUM, LogCleanerIoMaxBytesPerSecondDoc) - .define(LogCleanerDedupeBufferSizeProp, LONG, Defaults.LOG_CLEANER_DEDUPE_BUFFER_SIZE, MEDIUM, LogCleanerDedupeBufferSizeDoc) - .define(LogCleanerIoBufferSizeProp, INT, Defaults.LOG_CLEANER_IO_BUFFER_SIZE, atLeast(0), MEDIUM, LogCleanerIoBufferSizeDoc) - .define(LogCleanerDedupeBufferLoadFactorProp, DOUBLE, Defaults.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR, MEDIUM, LogCleanerDedupeBufferLoadFactorDoc) - .define(LogCleanerBackoffMsProp, LONG, Defaults.LOG_CLEANER_BACKOFF_MS, atLeast(0), MEDIUM, LogCleanerBackoffMsDoc) - .define(LogCleanerMinCleanRatioProp, DOUBLE, LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO, between(0, 1), MEDIUM, LogCleanerMinCleanRatioDoc) - .define(LogCleanerEnableProp, BOOLEAN, Defaults.LOG_CLEANER_ENABLE, MEDIUM, LogCleanerEnableDoc) - .define(LogCleanerDeleteRetentionMsProp, LONG, LogConfig.DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM, LogCleanerDeleteRetentionMsDoc) - .define(LogCleanerMinCompactionLagMsProp, LONG, LogConfig.DEFAULT_MIN_COMPACTION_LAG_MS, atLeast(0), MEDIUM, LogCleanerMinCompactionLagMsDoc) - .define(LogCleanerMaxCompactionLagMsProp, LONG, LogConfig.DEFAULT_MAX_COMPACTION_LAG_MS, atLeast(1), MEDIUM, LogCleanerMaxCompactionLagMsDoc) + .define(CleanerConfig.LOG_CLEANER_THREADS_PROP, INT, CleanerConfig.LOG_CLEANER_THREADS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_THREADS_DOC) + .define(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, DOUBLE, CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND, MEDIUM, CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_DOC) + .define(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, LONG, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE, MEDIUM, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_DOC) + .define(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, INT, CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_DOC) + .define(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, DOUBLE, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR, MEDIUM, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_DOC) + .define(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, LONG, CleanerConfig.LOG_CLEANER_BACKOFF_MS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_BACKOFF_MS_DOC) + .define(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP, DOUBLE, LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO, between(0, 1), MEDIUM, CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_DOC) + .define(CleanerConfig.LOG_CLEANER_ENABLE_PROP, BOOLEAN, CleanerConfig.LOG_CLEANER_ENABLE, MEDIUM, CleanerConfig.LOG_CLEANER_ENABLE_DOC) + .define(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, LONG, LogConfig.DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_DOC) + .define(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP, LONG, LogConfig.DEFAULT_MIN_COMPACTION_LAG_MS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_DOC) + .define(CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP, LONG, LogConfig.DEFAULT_MAX_COMPACTION_LAG_MS, atLeast(1), MEDIUM, CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_DOC) .define(LogIndexSizeMaxBytesProp, INT, LogConfig.DEFAULT_SEGMENT_INDEX_BYTES, atLeast(4), MEDIUM, LogIndexSizeMaxBytesDoc) .define(LogIndexIntervalBytesProp, INT, LogConfig.DEFAULT_INDEX_INTERVAL_BYTES, atLeast(0), MEDIUM, LogIndexIntervalBytesDoc) .define(LogFlushIntervalMessagesProp, LONG, LogConfig.DEFAULT_FLUSH_MESSAGES_INTERVAL, atLeast(1), HIGH, LogFlushIntervalMessagesDoc) @@ -1821,7 +1791,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val logDirs = CoreUtils.parseCsvList(Option(getString(KafkaConfig.LogDirsProp)).getOrElse(getString(KafkaConfig.LogDirProp))) def logSegmentBytes = getInt(KafkaConfig.LogSegmentBytesProp) def logFlushIntervalMessages = getLong(KafkaConfig.LogFlushIntervalMessagesProp) - val logCleanerThreads = getInt(KafkaConfig.LogCleanerThreadsProp) + val logCleanerThreads = getInt(CleanerConfig.LOG_CLEANER_THREADS_PROP) def numRecoveryThreadsPerDataDir = getInt(KafkaConfig.NumRecoveryThreadsPerDataDirProp) val logFlushSchedulerIntervalMs = getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp) val logFlushOffsetCheckpointIntervalMs = getInt(KafkaConfig.LogFlushOffsetCheckpointIntervalMsProp).toLong @@ -1831,16 +1801,16 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val offsetsRetentionMinutes = getInt(KafkaConfig.OffsetsRetentionMinutesProp) val offsetsRetentionCheckIntervalMs = getLong(KafkaConfig.OffsetsRetentionCheckIntervalMsProp) def logRetentionBytes = getLong(KafkaConfig.LogRetentionBytesProp) - val logCleanerDedupeBufferSize = getLong(KafkaConfig.LogCleanerDedupeBufferSizeProp) - val logCleanerDedupeBufferLoadFactor = getDouble(KafkaConfig.LogCleanerDedupeBufferLoadFactorProp) - val logCleanerIoBufferSize = getInt(KafkaConfig.LogCleanerIoBufferSizeProp) - val logCleanerIoMaxBytesPerSecond = getDouble(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp) - def logCleanerDeleteRetentionMs = getLong(KafkaConfig.LogCleanerDeleteRetentionMsProp) - def logCleanerMinCompactionLagMs = getLong(KafkaConfig.LogCleanerMinCompactionLagMsProp) - def logCleanerMaxCompactionLagMs = getLong(KafkaConfig.LogCleanerMaxCompactionLagMsProp) - val logCleanerBackoffMs = getLong(KafkaConfig.LogCleanerBackoffMsProp) - def logCleanerMinCleanRatio = getDouble(KafkaConfig.LogCleanerMinCleanRatioProp) - val logCleanerEnable = getBoolean(KafkaConfig.LogCleanerEnableProp) + val logCleanerDedupeBufferSize = getLong(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP) + val logCleanerDedupeBufferLoadFactor = getDouble(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP) + val logCleanerIoBufferSize = getInt(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP) + val logCleanerIoMaxBytesPerSecond = getDouble(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP) + def logCleanerDeleteRetentionMs = getLong(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP) + def logCleanerMinCompactionLagMs = getLong(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP) + def logCleanerMaxCompactionLagMs = getLong(CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP) + val logCleanerBackoffMs = getLong(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP) + def logCleanerMinCleanRatio = getDouble(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP) + val logCleanerEnable = getBoolean(CleanerConfig.LOG_CLEANER_ENABLE_PROP) def logIndexSizeMaxBytes = getInt(KafkaConfig.LogIndexSizeMaxBytesProp) def logIndexIntervalBytes = getInt(KafkaConfig.LogIndexIntervalBytesProp) def logDeleteDelayMs = getLong(KafkaConfig.LogDeleteDelayMsProp) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index c5e4218b29..3baea3589b 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -618,7 +618,7 @@ class KafkaServer( } } } - socketServer.enableRequestProcessing(authorizerFutures) + val enableRequestProcessingFuture = socketServer.enableRequestProcessing(authorizerFutures) // Block here until all the authorizer futures are complete try { CompletableFuture.allOf(authorizerFutures.values.toSeq: _*).join() @@ -626,6 +626,13 @@ class KafkaServer( case t: Throwable => throw new RuntimeException("Received a fatal error while " + "waiting for all of the authorizer futures to be completed.", t) } + // Wait for all the SocketServer ports to be open, and the Acceptors to be started. + try { + enableRequestProcessingFuture.join() + } catch { + case t: Throwable => throw new RuntimeException("Received a fatal error while " + + "waiting for the SocketServer Acceptors to be started.", t) + } _brokerState = BrokerState.RUNNING shutdownLatch = new CountDownLatch(1) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index e48eb84f78..37f3bd0120 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -797,6 +797,7 @@ class ReplicaManager(val config: KafkaConfig, * @param requestLocal container for the stateful instances scoped to this request -- this must correspond to the * thread calling this method * @param actionQueue the action queue to use. ReplicaManager#defaultActionQueue is used by default. + * @param supportedOperation determines the supported Operation based on the client's Request api version * * The responseCallback is wrapped so that it is scheduled on a request handler thread. There, it should be called with * that request handler thread's thread local and not the one supplied to this method. @@ -809,7 +810,8 @@ class ReplicaManager(val config: KafkaConfig, responseCallback: Map[TopicPartition, PartitionResponse] => Unit, recordValidationStatsCallback: Map[TopicPartition, RecordValidationStats] => Unit = _ => (), requestLocal: RequestLocal = RequestLocal.NoCaching, - actionQueue: ActionQueue = this.defaultActionQueue): Unit = { + actionQueue: ActionQueue = this.defaultActionQueue, + supportedOperation: SupportedOperation): Unit = { val transactionalProducerInfo = mutable.HashSet[(Long, Short)]() val topicPartitionBatchInfo = mutable.Map[TopicPartition, Int]() @@ -832,7 +834,12 @@ class ReplicaManager(val config: KafkaConfig, val customException = error match { case Errors.INVALID_TXN_STATE => Some(error.exception("Partition was not added to the transaction")) + // Transaction verification can fail with a retriable error that older clients may not + // retry correctly. Translate these to an error which will cause such clients to retry + // the produce request. We pick `NOT_ENOUGH_REPLICAS` because it does not trigger a + // metadata refresh. case Errors.CONCURRENT_TRANSACTIONS | + Errors.NETWORK_EXCEPTION | Errors.COORDINATOR_LOAD_IN_PROGRESS | Errors.COORDINATOR_NOT_AVAILABLE | Errors.NOT_COORDINATOR => Some(new NotEnoughReplicasException( @@ -891,7 +898,8 @@ class ReplicaManager(val config: KafkaConfig, // 4. wrap the callback: after real callback, set the hasInflight = false and re-run the pending tasks. verifyTransactionCallbackWrapper(verification, postVerificationCallback), requestLocal - ) + ), + supportedOperation ) } // 2. ensure only one request is verifying transaction @@ -1006,12 +1014,13 @@ class ReplicaManager(val config: KafkaConfig, /** * - * @param topicPartition the topic partition to maybe verify - * @param transactionalId the transactional id for the transaction - * @param producerId the producer id for the producer writing to the transaction - * @param producerEpoch the epoch of the producer writing to the transaction - * @param baseSequence the base sequence of the first record in the batch we are trying to append - * @param callback the method to execute once the verification is either completed or returns an error + * @param topicPartition the topic partition to maybe verify + * @param transactionalId the transactional id for the transaction + * @param producerId the producer id for the producer writing to the transaction + * @param producerEpoch the epoch of the producer writing to the transaction + * @param baseSequence the base sequence of the first record in the batch we are trying to append + * @param callback the method to execute once the verification is either completed or returns an error + * @param supportedOperation determines the supported operation based on the client's Request API version * * When the verification returns, the callback will be supplied the error if it exists or Errors.NONE. * If the verification guard exists, it will also be supplied. Otherwise the SENTINEL verification guard will be returned. @@ -1023,7 +1032,8 @@ class ReplicaManager(val config: KafkaConfig, producerId: Long, producerEpoch: Short, baseSequence: Int, - callback: ((Errors, VerificationGuard)) => Unit + callback: ((Errors, VerificationGuard)) => Unit, + supportedOperation: SupportedOperation ): Unit = { def generalizedCallback(results: (Map[TopicPartition, Errors], Map[TopicPartition, VerificationGuard])): Unit = { val (preAppendErrors, verificationGuards) = results @@ -1038,7 +1048,8 @@ class ReplicaManager(val config: KafkaConfig, transactionalId, producerId, producerEpoch, - generalizedCallback + generalizedCallback, + supportedOperation ) } @@ -1049,6 +1060,7 @@ class ReplicaManager(val config: KafkaConfig, * @param producerId the producer id for the producer writing to the transaction * @param producerEpoch the epoch of the producer writing to the transaction * @param callback the method to execute once the verification is either completed or returns an error + * @param supportedOperation determines the supported operation based on the client's Request API version * * When the verification returns, the callback will be supplied the errors per topic partition if there were errors. * The callback will also be supplied the verification guards per partition if they exist. It is possible to have an @@ -1060,7 +1072,8 @@ class ReplicaManager(val config: KafkaConfig, transactionalId: String, producerId: Long, producerEpoch: Short, - callback: ((Map[TopicPartition, Errors], Map[TopicPartition, VerificationGuard])) => Unit + callback: ((Map[TopicPartition, Errors], Map[TopicPartition, VerificationGuard])) => Unit, + supportedOperation: SupportedOperation ): Unit = { // Skip verification if the request is not transactional or transaction verification is disabled. if (transactionalId == null || @@ -1106,7 +1119,8 @@ class ReplicaManager(val config: KafkaConfig, producerId = producerId, producerEpoch = producerEpoch, topicPartitions = verificationGuards.keys.toSeq, - callback = invokeCallback + callback = invokeCallback, + supportedOperation = supportedOperation )) } diff --git a/core/src/main/scala/kafka/server/streamaspect/ElasticKafkaApis.scala b/core/src/main/scala/kafka/server/streamaspect/ElasticKafkaApis.scala index ec5e815b90..ebf27df455 100644 --- a/core/src/main/scala/kafka/server/streamaspect/ElasticKafkaApis.scala +++ b/core/src/main/scala/kafka/server/streamaspect/ElasticKafkaApis.scala @@ -339,6 +339,7 @@ class ElasticKafkaApis( sendResponseCallback(Map.empty) else { val internalTopicsAllowed = request.header.clientId == AdminUtils.ADMIN_CLIENT_ID + val supportedOperation = if (request.header.apiVersion > 10) genericError else defaultError def doAppendRecords(): Unit = { // call the replica manager to append messages to the replicas @@ -350,7 +351,8 @@ class ElasticKafkaApis( entriesPerPartition = authorizedRequestInfo, responseCallback = sendResponseCallback, recordValidationStatsCallback = processingStatsCallback, - requestLocal = requestLocal) + requestLocal = requestLocal, + supportedOperation = supportedOperation) // if the request is put into the purgatory, it will have a held reference and hence cannot be garbage collected; // hence we clear its data here in order to let GC reclaim its memory since it is already appended to log diff --git a/core/src/main/scala/kafka/utils/ToolsUtils.scala b/core/src/main/scala/kafka/utils/ToolsUtils.scala index edd79c0003..ffb6214d0b 100644 --- a/core/src/main/scala/kafka/utils/ToolsUtils.scala +++ b/core/src/main/scala/kafka/utils/ToolsUtils.scala @@ -69,7 +69,7 @@ object ToolsUtils { /** * This is a simple wrapper around `CommandLineUtils.printUsageAndExit`. * It is needed for tools migration (KAFKA-14525), as there is no Java equivalent for return type `Nothing`. - * Can be removed once [[kafka.admin.ConsumerGroupCommand]] and [[kafka.tools.ConsoleProducer]] are migrated. + * Can be removed once [[kafka.tools.ConsoleProducer]] are migrated. * * @param parser Command line options parser. * @param message Error message. diff --git a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala index ee960bd35f..e69719e750 100644 --- a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala +++ b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala @@ -27,6 +27,7 @@ import org.apache.kafka.common.errors.ControllerMovedException import org.apache.kafka.common.metadata._ import org.apache.kafka.common.resource.ResourcePattern import org.apache.kafka.common.security.scram.ScramCredential +import org.apache.kafka.common.utils.Sanitizer import org.apache.kafka.common.{TopicIdPartition, Uuid} import org.apache.kafka.metadata.DelegationTokenData import org.apache.kafka.metadata.PartitionRegistration @@ -226,6 +227,9 @@ class ZkMigrationClient( entityDataList: util.List[ClientQuotaRecord.EntityData], quotas: util.Map[String, lang.Double] ): Unit = { + entityDataList.forEach(entityData => { + entityData.setEntityName(Sanitizer.desanitize(entityData.entityName())) + }) val batch = new util.ArrayList[ApiMessageAndVersion]() quotas.forEach((key, value) => { batch.add(new ApiMessageAndVersion(new ClientQuotaRecord() diff --git a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala index 846a599875..dffdc138aa 100644 --- a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala +++ b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala @@ -228,8 +228,8 @@ class ZkConfigMigrationClient( val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(requests, state) if (responses.head.resultCode.equals(Code.NONODE)) { - // Not fatal. - error(s"Did not delete $configResource since the node did not exist.") + // Not fatal. This is expected in the case this is a topic config and we delete the topic + debug(s"Did not delete $configResource since the node did not exist.") state } else if (responses.head.resultCode.equals(Code.OK)) { // Write the notification znode if our update was successful diff --git a/core/src/test/java/kafka/server/handlers/DescribeTopicPartitionsRequestHandlerTest.java b/core/src/test/java/kafka/server/handlers/DescribeTopicPartitionsRequestHandlerTest.java index 40666a290d..88cb82644c 100644 --- a/core/src/test/java/kafka/server/handlers/DescribeTopicPartitionsRequestHandlerTest.java +++ b/core/src/test/java/kafka/server/handlers/DescribeTopicPartitionsRequestHandlerTest.java @@ -76,6 +76,7 @@ import java.util.stream.Collectors; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -469,7 +470,7 @@ void testDescribeTopicPartitionsRequestWithEdgeCases() { try { handler.handleDescribeTopicPartitionsRequest(buildRequest(describeTopicPartitionsRequest, plaintextListener)); } catch (Exception e) { - assertTrue(e instanceof InvalidRequestException, e.getMessage()); + assertInstanceOf(InvalidRequestException.class, e, e.getMessage()); } } diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index 8759d2149b..0c77441827 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -41,6 +41,7 @@ import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.fault.FaultHandler; import org.apache.kafka.server.fault.MockFaultHandler; +import org.apache.kafka.storage.internals.log.CleanerConfig; import org.apache.kafka.test.TestUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -192,7 +193,7 @@ private KafkaConfig createNodeConfig(TestKitNode node) { props.put(RaftConfig.QUORUM_VOTERS_CONFIG, uninitializedQuorumVotersString); // reduce log cleaner offset map memory usage - props.put(KafkaConfig$.MODULE$.LogCleanerDedupeBufferSizeProp(), "2097152"); + props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152"); // Add associated broker node property overrides if (brokerNode != null) { diff --git a/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala b/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala index 333bd980a6..5362a1d5e3 100644 --- a/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala @@ -19,82 +19,255 @@ package kafka.admin import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig +import kafka.utils.TestUtils.{createProducer, plaintextBootstrapServers} import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.admin._ import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.utils.Utils +import org.apache.kafka.common.config.TopicConfig +import org.apache.kafka.common.utils.{MockTime, Time, Utils} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource +import java.util.Properties import scala.collection.{Map, Seq} import scala.jdk.CollectionConverters._ class ListOffsetsIntegrationTest extends KafkaServerTestHarness { val topicName = "foo" + val topicNameWithCustomConfigs = "foo2" var adminClient: Admin = _ + var setOldMessageFormat: Boolean = false + val mockTime: Time = new MockTime(1) @BeforeEach override def setUp(testInfo: TestInfo): Unit = { super.setUp(testInfo) - createTopic(topicName, 1, 1.toShort) - produceMessages() + createTopicWithConfig(topicName, new Properties()) adminClient = Admin.create(Map[String, Object]( AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG -> bootstrapServers() ).asJava) } + override def brokerTime(brokerId: Int): Time = mockTime + @AfterEach override def tearDown(): Unit = { + setOldMessageFormat = false Utils.closeQuietly(adminClient, "ListOffsetsAdminClient") super.tearDown() } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) - def testEarliestOffset(quorum: String): Unit = { - val earliestOffset = runFetchOffsets(adminClient, OffsetSpec.earliest()) - assertEquals(0, earliestOffset.offset()) + def testThreeCompressedRecordsInOneBatch(quorum: String): Unit = { + produceMessagesInOneBatch("gzip") + verifyListOffsets() + + // test LogAppendTime case + setUpForLogAppendTimeCase() + produceMessagesInOneBatch("gzip", topicNameWithCustomConfigs) + // In LogAppendTime's case, the maxTimestampOffset should be the first message of the batch. + // So in this one batch test, it'll be the first offset 0 + verifyListOffsets(topic = topicNameWithCustomConfigs, 0) } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) - def testLatestOffset(quorum: String): Unit = { - val latestOffset = runFetchOffsets(adminClient, OffsetSpec.latest()) - assertEquals(3, latestOffset.offset()) + def testThreeNonCompressedRecordsInOneBatch(quorum: String): Unit = { + produceMessagesInOneBatch() + verifyListOffsets() + + // test LogAppendTime case + setUpForLogAppendTimeCase() + produceMessagesInOneBatch(topic=topicNameWithCustomConfigs) + // In LogAppendTime's case, if the timestamps are the same, we choose the offset of the first record + // thus, the maxTimestampOffset should be the first record of the batch. + // So in this one batch test, it'll be the first offset which is 0 + verifyListOffsets(topic = topicNameWithCustomConfigs, 0) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testThreeNonCompressedRecordsInSeparateBatch(quorum: String): Unit = { + produceMessagesInSeparateBatch() + verifyListOffsets() + + // test LogAppendTime case + setUpForLogAppendTimeCase() + produceMessagesInSeparateBatch(topic = topicNameWithCustomConfigs) + // In LogAppendTime's case, if the timestamp is different, it should be the last one + verifyListOffsets(topic = topicNameWithCustomConfigs, 2) + } + + // The message conversion test only run in ZK mode because KRaft mode doesn't support "inter.broker.protocol.version" < 3.0 + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk")) + def testThreeRecordsInOneBatchWithMessageConversion(quorum: String): Unit = { + createOldMessageFormatBrokers() + produceMessagesInOneBatch() + verifyListOffsets() + + // test LogAppendTime case + setUpForLogAppendTimeCase() + produceMessagesInOneBatch(topic = topicNameWithCustomConfigs) + // In LogAppendTime's case, the maxTimestampOffset should be the first message of the batch. + // So in this one batch test, it'll be the first offset 0 + verifyListOffsets(topic = topicNameWithCustomConfigs, 0) + } + + // The message conversion test only run in ZK mode because KRaft mode doesn't support "inter.broker.protocol.version" < 3.0 + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk")) + def testThreeRecordsInSeparateBatchWithMessageConversion(quorum: String): Unit = { + createOldMessageFormatBrokers() + produceMessagesInSeparateBatch() + verifyListOffsets() + + // test LogAppendTime case + setUpForLogAppendTimeCase() + produceMessagesInSeparateBatch(topic = topicNameWithCustomConfigs) + // In LogAppendTime's case, the maxTimestampOffset should be the first message of the batch. + // So in this separate batch test, it'll be the last offset 2 + verifyListOffsets(topic = topicNameWithCustomConfigs, 2) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testThreeRecordsInOneBatchHavingDifferentCompressionTypeWithServer(quorum: String): Unit = { + val props: Properties = new Properties() + props.setProperty(TopicConfig.COMPRESSION_TYPE_CONFIG, "lz4") + createTopicWithConfig(topicNameWithCustomConfigs, props) + produceMessagesInOneBatch(topic = topicNameWithCustomConfigs) + verifyListOffsets(topic = topicNameWithCustomConfigs) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testThreeRecordsInSeparateBatchHavingDifferentCompressionTypeWithServer(quorum: String): Unit = { + val props: Properties = new Properties() + props.setProperty(TopicConfig.COMPRESSION_TYPE_CONFIG, "lz4") + createTopicWithConfig(topicNameWithCustomConfigs, props) + produceMessagesInSeparateBatch(topic = topicNameWithCustomConfigs) + verifyListOffsets(topic = topicNameWithCustomConfigs) } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) - def testMaxTimestampOffset(quorum: String): Unit = { - val maxTimestampOffset = runFetchOffsets(adminClient, OffsetSpec.maxTimestamp()) - assertEquals(1, maxTimestampOffset.offset()) + def testThreeCompressedRecordsInSeparateBatch(quorum: String): Unit = { + produceMessagesInSeparateBatch("gzip") + verifyListOffsets() + + // test LogAppendTime case + setUpForLogAppendTimeCase() + produceMessagesInSeparateBatch("gzip", topicNameWithCustomConfigs) + // In LogAppendTime's case, the maxTimestampOffset should be the first message of the batch. + // So in this separate batch test, it'll be the last offset 2 + verifyListOffsets(topic = topicNameWithCustomConfigs, 2) + } + + private def setUpForLogAppendTimeCase(): Unit = { + val props: Properties = new Properties() + props.setProperty(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, "LogAppendTime") + createTopicWithConfig(topicNameWithCustomConfigs, props) + } + + private def createOldMessageFormatBrokers(): Unit = { + setOldMessageFormat = true + recreateBrokers(reconfigure = true, startup = true) + Utils.closeQuietly(adminClient, "ListOffsetsAdminClient") + adminClient = Admin.create(Map[String, Object]( + AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG -> bootstrapServers() + ).asJava) + } + + private def createTopicWithConfig(topic: String, props: Properties): Unit = { + createTopic(topic, 1, 1.toShort, topicConfig = props) + } + + private def verifyListOffsets(topic: String = topicName, expectedMaxTimestampOffset: Int = 1): Unit = { + val earliestOffset = runFetchOffsets(adminClient, OffsetSpec.earliest(), topic) + assertEquals(0, earliestOffset.offset()) + + val latestOffset = runFetchOffsets(adminClient, OffsetSpec.latest(), topic) + assertEquals(3, latestOffset.offset()) + + val maxTimestampOffset = runFetchOffsets(adminClient, OffsetSpec.maxTimestamp(), topic) + assertEquals(expectedMaxTimestampOffset, maxTimestampOffset.offset()) } private def runFetchOffsets(adminClient: Admin, - offsetSpec: OffsetSpec): ListOffsetsResult.ListOffsetsResultInfo = { - val tp = new TopicPartition(topicName, 0) + offsetSpec: OffsetSpec, + topic: String): ListOffsetsResult.ListOffsetsResultInfo = { + val tp = new TopicPartition(topic, 0) adminClient.listOffsets(Map( tp -> offsetSpec ).asJava, new ListOffsetsOptions()).all().get().get(tp) } - def produceMessages(): Unit = { + private def produceMessagesInOneBatch(compressionType: String = "none", topic: String = topicName): Unit = { val records = Seq( - new ProducerRecord[Array[Byte], Array[Byte]](topicName, 0, 100L, - null, new Array[Byte](10000)), - new ProducerRecord[Array[Byte], Array[Byte]](topicName, 0, 999L, - null, new Array[Byte](10000)), - new ProducerRecord[Array[Byte], Array[Byte]](topicName, 0, 200L, - null, new Array[Byte](10000)), + new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, 100L, + null, new Array[Byte](10)), + new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, 999L, + null, new Array[Byte](10)), + new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, 200L, + null, new Array[Byte](10)), ) - TestUtils.produceMessages(brokers, records, -1) + // create a producer with large linger.ms and enough batch.size (default is enough for three 10 bytes records), + // so that we can confirm all records will be accumulated in producer until we flush them into one batch. + val producer = createProducer( + plaintextBootstrapServers(brokers), + deliveryTimeoutMs = Int.MaxValue, + lingerMs = Int.MaxValue, + compressionType = compressionType) + + try { + val futures = records.map(producer.send) + producer.flush() + futures.foreach(_.get) + } finally { + producer.close() + } + } + + private def produceMessagesInSeparateBatch(compressionType: String = "none", topic: String = topicName): Unit = { + val records = Seq(new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, 100L, + null, new Array[Byte](10))) + val records2 = Seq(new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, 999L, + null, new Array[Byte](10))) + val records3 = Seq(new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, 200L, + null, new Array[Byte](10))) + + val producer = createProducer( + plaintextBootstrapServers(brokers), + compressionType = compressionType) + try { + val futures = records.map(producer.send) + futures.foreach(_.get) + // advance the server time after each record sent to make sure the time changed when appendTime is used + mockTime.sleep(100) + val futures2 = records2.map(producer.send) + futures2.foreach(_.get) + mockTime.sleep(100) + val futures3 = records3.map(producer.send) + futures3.foreach(_.get) + } finally { + producer.close() + } } - def generateConfigs: Seq[KafkaConfig] = - TestUtils.createBrokerConfigs(1, zkConnectOrNull).map(KafkaConfig.fromProps) + def generateConfigs: Seq[KafkaConfig] = { + TestUtils.createBrokerConfigs(1, zkConnectOrNull).map{ props => + if (setOldMessageFormat) { + props.setProperty("log.message.format.version", "0.10.0") + props.setProperty("inter.broker.protocol.version", "0.10.0") + } + props + }.map(KafkaConfig.fromProps) + } } diff --git a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala new file mode 100644 index 0000000000..b3e1ba9a64 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala @@ -0,0 +1,130 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package kafka.api + +import kafka.security.authorizer.AclAuthorizer +import kafka.security.authorizer.AclEntry.WildcardHost +import kafka.server.{BaseRequestTest, KafkaConfig} +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.acl.AccessControlEntry +import org.apache.kafka.common.acl.AclOperation.CLUSTER_ACTION +import org.apache.kafka.common.acl.AclPermissionType.ALLOW +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs +import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.resource.PatternType.LITERAL +import org.apache.kafka.common.resource.{Resource, ResourcePattern} +import org.apache.kafka.common.resource.ResourceType.{CLUSTER, GROUP, TOPIC, TRANSACTIONAL_ID} +import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal} +import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder +import org.apache.kafka.metadata.authorizer.StandardAuthorizer +import org.junit.jupiter.api.{BeforeEach, TestInfo} + +import java.util.Properties + +object AbstractAuthorizerIntegrationTest { + val BrokerPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "broker") + val ClientPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "client") + + val BrokerListenerName = "BROKER" + val ClientListenerName = "CLIENT" + val ControllerListenerName = "CONTROLLER" + + class PrincipalBuilder extends DefaultKafkaPrincipalBuilder(null, null) { + override def build(context: AuthenticationContext): KafkaPrincipal = { + context.listenerName match { + case BrokerListenerName | ControllerListenerName => BrokerPrincipal + case ClientListenerName => ClientPrincipal + case listenerName => throw new IllegalArgumentException(s"No principal mapped to listener $listenerName") + } + } + } +} + +/** + * Abstract authorizer test to be used both in scala and java tests of authorizer. + */ +class AbstractAuthorizerIntegrationTest extends BaseRequestTest { + import AbstractAuthorizerIntegrationTest._ + + override def interBrokerListenerName: ListenerName = new ListenerName(BrokerListenerName) + override def listenerName: ListenerName = new ListenerName(ClientListenerName) + override def brokerCount: Int = 1 + + def clientPrincipal: KafkaPrincipal = ClientPrincipal + def brokerPrincipal: KafkaPrincipal = BrokerPrincipal + + val clientPrincipalString: String = clientPrincipal.toString + + val brokerId: Integer = 0 + val topic = "topic" + val topicPattern = "topic.*" + val transactionalId = "transactional.id" + val producerId = 83392L + val part = 0 + val correlationId = 0 + val clientId = "client-Id" + val tp = new TopicPartition(topic, part) + val logDir = "logDir" + val group = "my-group" + val protocolType = "consumer" + val protocolName = "consumer-range" + val clusterResource = new ResourcePattern(CLUSTER, Resource.CLUSTER_NAME, LITERAL) + val topicResource = new ResourcePattern(TOPIC, topic, LITERAL) + val groupResource = new ResourcePattern(GROUP, group, LITERAL) + val transactionalIdResource = new ResourcePattern(TRANSACTIONAL_ID, transactionalId, LITERAL) + + producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "1") + producerConfig.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false") + producerConfig.setProperty(ProducerConfig.MAX_BLOCK_MS_CONFIG, "50000") + consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, group) + + override def brokerPropertyOverrides(properties: Properties): Unit = { + properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) + addNodeProperties(properties) + } + + override def kraftControllerConfigs(): collection.Seq[Properties] = { + val controllerConfigs = super.kraftControllerConfigs() + controllerConfigs.foreach(addNodeProperties) + controllerConfigs + } + + private def addNodeProperties(properties: Properties): Unit = { + if (isKRaftTest()) { + properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + properties.put(StandardAuthorizer.SUPER_USERS_CONFIG, BrokerPrincipal.toString) + } else { + properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) + } + + properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1") + properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") + properties.put(KafkaConfig.TransactionsTopicPartitionsProp, "1") + properties.put(KafkaConfig.TransactionsTopicReplicationFactorProp, "1") + properties.put(KafkaConfig.TransactionsTopicMinISRProp, "1") + properties.put(KafkaConfig.UnstableApiVersionsEnableProp, "true") + properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[PrincipalBuilder].getName) + } + + @BeforeEach + override def setUp(testInfo: TestInfo): Unit = { + doSetup(testInfo, createOffsetsTopic = false) + + // Allow inter-broker communication + addAndVerifyAcls(Set(new AccessControlEntry(brokerPrincipal.toString, WildcardHost, CLUSTER_ACTION, ALLOW)), clusterResource) + + createOffsetsTopic(listenerName = interBrokerListenerName) + } +} diff --git a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala index ee9d8426ee..78d51329b7 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala @@ -82,6 +82,80 @@ abstract class AbstractConsumerTest extends BaseRequestTest { createTopic(topic, 2, brokerCount, adminClientConfig = this.adminClientConfig) } + def awaitAssignment(consumer: Consumer[_, _], expectedAssignment: Set[TopicPartition]) + : Unit = { + TestUtils.pollUntilTrue(consumer, () => consumer.assignment() == expectedAssignment.asJava, + s"Timed out while awaiting expected assignment $expectedAssignment. " + + s"The current assignment is ${consumer.assignment()}") + } + + def awaitNonEmptyRecords[K, V](consumer: Consumer[K, V], partition: TopicPartition): ConsumerRecords[K, V] = { + TestUtils.pollRecordsUntilTrue(consumer, (polledRecords: ConsumerRecords[K, V]) => { + if (polledRecords.records(partition).asScala.nonEmpty) + return polledRecords + false + }, s"Consumer did not consume any messages for partition $partition before timeout.") + throw new IllegalStateException("Should have timed out before reaching here") + } + + /** + * Creates 'consumerCount' consumers and consumer pollers, one per consumer; subscribes consumers to + * 'topicsToSubscribe' topics, waits until consumers get topics assignment. + * + * When the function returns, consumer pollers will continue to poll until shutdown is called on every poller. + * + * @param consumerCount number of consumers to create + * @param topicsToSubscribe topics to which consumers will subscribe to + * @param subscriptions set of all topic partitions + * @return collection of created consumers and collection of corresponding consumer pollers + */ + def createConsumerGroupAndWaitForAssignment(consumerCount: Int, + topicsToSubscribe: List[String], + subscriptions: Set[TopicPartition]): (Buffer[Consumer[Array[Byte], Array[Byte]]], Buffer[ConsumerAssignmentPoller]) = { + assertTrue(consumerCount <= subscriptions.size) + val consumerGroup = Buffer[Consumer[Array[Byte], Array[Byte]]]() + for (_ <- 0 until consumerCount) + consumerGroup += createConsumer() + + // create consumer pollers, wait for assignment and validate it + val consumerPollers = subscribeConsumers(consumerGroup, topicsToSubscribe) + (consumerGroup, consumerPollers) + } + + /** + * Creates consumer pollers corresponding to a given consumer group, one per consumer; subscribes consumers to + * 'topicsToSubscribe' topics, waits until consumers get topics assignment. + * + * When the function returns, consumer pollers will continue to poll until shutdown is called on every poller. + * + * @param consumerGroup consumer group + * @param topicsToSubscribe topics to which consumers will subscribe to + * @return collection of consumer pollers + */ + def subscribeConsumers(consumerGroup: mutable.Buffer[Consumer[Array[Byte], Array[Byte]]], + topicsToSubscribe: List[String]): mutable.Buffer[ConsumerAssignmentPoller] = { + val consumerPollers = mutable.Buffer[ConsumerAssignmentPoller]() + for (consumer <- consumerGroup) + consumerPollers += subscribeConsumerAndStartPolling(consumer, topicsToSubscribe) + consumerPollers + } + + def changeConsumerGroupSubscriptionAndValidateAssignment(consumerPollers: Buffer[ConsumerAssignmentPoller], + topicsToSubscribe: List[String], + subscriptions: Set[TopicPartition]): Unit = { + for (poller <- consumerPollers) + poller.subscribe(topicsToSubscribe) + + // since subscribe call to poller does not actually call consumer subscribe right away, wait + // until subscribe is called on all consumers + TestUtils.waitUntilTrue(() => { + consumerPollers.forall { poller => poller.isSubscribeRequestProcessed } + }, s"Failed to call subscribe on all consumers in the group for subscription $subscriptions", 1000L) + + validateGroupAssignment(consumerPollers, subscriptions, + Some(s"Did not get valid assignment for partitions ${subscriptions.asJava} after we changed subscription")) + } + protected class TestConsumerReassignmentListener extends ConsumerRebalanceListener { var callsToAssigned = 0 var callsToRevoked = 0 diff --git a/core/src/test/scala/integration/kafka/api/AbstractSaslTest.scala b/core/src/test/scala/integration/kafka/api/AbstractSaslTest.scala new file mode 100644 index 0000000000..4bef3422a2 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/AbstractSaslTest.scala @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package kafka.api + +/** + * Abstract Sasl test to be used both in scala and java tests of SASL. + * Separate class required to overcome issues related to usage of scala trait in java code. + * @see SaslClientsWithInvalidCredentialsTest + */ +abstract class AbstractSaslTest extends IntegrationTestHarness with SaslSetup { +} + diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index c399b46450..ded470eba8 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -18,10 +18,8 @@ import java.util import java.util.concurrent.ExecutionException import java.util.regex.Pattern import java.util.{Collections, Optional, Properties} -import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService} -import kafka.security.authorizer.{AclAuthorizer, AclEntry} +import kafka.security.authorizer.AclEntry import kafka.security.authorizer.AclEntry.WildcardHost -import kafka.server.{BaseRequestTest, KafkaConfig} import kafka.utils.{TestInfoUtils, TestUtils} import kafka.utils.TestUtils.waitUntilTrue import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, NewTopic} @@ -30,7 +28,6 @@ import org.apache.kafka.clients.producer._ import org.apache.kafka.common.acl.AclOperation._ import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY} import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBindingFilter, AclOperation, AclPermissionType} -import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, TopicConfig} import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic.GROUP_METADATA_TOPIC_NAME @@ -53,14 +50,11 @@ import org.apache.kafka.common.requests._ import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.resource.ResourceType._ import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourcePatternFilter, ResourceType} -import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, SecurityProtocol} -import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder +import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{ElectionType, IsolationLevel, KafkaException, Node, TopicPartition, Uuid, requests} -import org.apache.kafka.metadata.authorizer.StandardAuthorizer import org.apache.kafka.test.{TestUtils => JTestUtils} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{CsvSource, ValueSource} @@ -72,55 +66,7 @@ import scala.annotation.nowarn import scala.collection.mutable import scala.jdk.CollectionConverters._ -object AuthorizerIntegrationTest { - val BrokerPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "broker") - val ClientPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "client") - - val BrokerListenerName = "BROKER" - val ClientListenerName = "CLIENT" - val ControllerListenerName = "CONTROLLER" - - class PrincipalBuilder extends DefaultKafkaPrincipalBuilder(null, null) { - override def build(context: AuthenticationContext): KafkaPrincipal = { - context.listenerName match { - case BrokerListenerName | ControllerListenerName => BrokerPrincipal - case ClientListenerName => ClientPrincipal - case listenerName => throw new IllegalArgumentException(s"No principal mapped to listener $listenerName") - } - } - } -} - -class AuthorizerIntegrationTest extends BaseRequestTest { - import AuthorizerIntegrationTest._ - - override def interBrokerListenerName: ListenerName = new ListenerName(BrokerListenerName) - override def listenerName: ListenerName = new ListenerName(ClientListenerName) - override def brokerCount: Int = 1 - - def clientPrincipal: KafkaPrincipal = ClientPrincipal - def brokerPrincipal: KafkaPrincipal = BrokerPrincipal - - val clientPrincipalString: String = clientPrincipal.toString - - val brokerId: Integer = 0 - val topic = "topic" - val topicPattern = "topic.*" - val transactionalId = "transactional.id" - val producerId = 83392L - val part = 0 - val correlationId = 0 - val clientId = "client-Id" - val tp = new TopicPartition(topic, part) - val logDir = "logDir" - val group = "my-group" - val protocolType = "consumer" - val protocolName = "consumer-range" - val clusterResource = new ResourcePattern(CLUSTER, Resource.CLUSTER_NAME, LITERAL) - val topicResource = new ResourcePattern(TOPIC, topic, LITERAL) - val groupResource = new ResourcePattern(GROUP, group, LITERAL) - val transactionalIdResource = new ResourcePattern(TRANSACTIONAL_ID, transactionalId, LITERAL) - +class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { val groupReadAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW))) val groupDescribeAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE, ALLOW))) val groupDeleteAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DELETE, ALLOW))) @@ -143,40 +89,6 @@ class AuthorizerIntegrationTest extends BaseRequestTest { val numRecords = 1 - producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "1") - producerConfig.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false") - producerConfig.setProperty(ProducerConfig.MAX_BLOCK_MS_CONFIG, "50000") - consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, group) - - override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) - addNodeProperties(properties) - } - - override def kraftControllerConfigs(): collection.Seq[Properties] = { - val controllerConfigs = super.kraftControllerConfigs() - controllerConfigs.foreach(addNodeProperties) - controllerConfigs - } - - private def addNodeProperties(properties: Properties): Unit = { - if (isKRaftTest()) { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) - properties.put(StandardAuthorizer.SUPER_USERS_CONFIG, BrokerPrincipal.toString) - } else { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) - } - - properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1") - properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") - properties.put(KafkaConfig.TransactionsTopicPartitionsProp, "1") - properties.put(KafkaConfig.TransactionsTopicReplicationFactorProp, "1") - properties.put(KafkaConfig.TransactionsTopicMinISRProp, "1") - properties.put(KafkaConfig.UnstableApiVersionsEnableProp, "true") - properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[PrincipalBuilder].getName) - } - - val requestKeyToError = (topicNames: Map[Uuid, String], version: Short) => Map[ApiKeys, Nothing => Errors]( ApiKeys.METADATA -> ((resp: requests.MetadataResponse) => resp.errors.asScala.find(_._1 == topic).getOrElse(("test", Errors.NONE))._2), ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => { @@ -333,16 +245,6 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ApiKeys.DESCRIBE_TRANSACTIONS -> transactionalIdDescribeAcl ) - @BeforeEach - override def setUp(testInfo: TestInfo): Unit = { - doSetup(testInfo, createOffsetsTopic = false) - - // Allow inter-broker communication - addAndVerifyAcls(Set(new AccessControlEntry(brokerPrincipal.toString, WildcardHost, CLUSTER_ACTION, ALLOW)), clusterResource) - - createOffsetsTopic(listenerName = interBrokerListenerName) - } - private def createMetadataRequest(allowAutoTopicCreation: Boolean) = { new requests.MetadataRequest.Builder(List(topic).asJava, allowAutoTopicCreation).build() } @@ -1703,20 +1605,6 @@ class AuthorizerIntegrationTest extends BaseRequestTest { createAdminClient().describeConsumerGroups(Seq(group).asJava).describedGroups().get(group).get() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeGroupCliWithGroupDescribe(quorum: String): Unit = { - createTopicWithBrokerPrincipal(topic) - addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE, ALLOW)), groupResource) - addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, DESCRIBE, ALLOW)), topicResource) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val opts = new ConsumerGroupCommandOptions(cgcArgs) - val consumerGroupService = new ConsumerGroupService(opts) - consumerGroupService.describeGroups() - consumerGroupService.close() - } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testListGroupApiWithAndWithoutListGroupAcls(quorum: String): Unit = { diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index bb3259baf9..2015983094 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -26,6 +26,7 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, MethodSource} +import java.util import java.util.Properties import java.util.concurrent.atomic.AtomicInteger import scala.jdk.CollectionConverters._ @@ -117,11 +118,12 @@ object BaseConsumerTest { // * KRaft with the new group coordinator enabled and the classic group protocol // * KRaft with the new group coordinator enabled and the consumer group protocol def getTestQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = { - java.util.stream.Stream.of( + util.Arrays.stream(Array( Arguments.of("zk", "classic"), Arguments.of("kraft", "classic"), Arguments.of("kraft+kip848", "classic"), - Arguments.of("kraft+kip848", "consumer")) + Arguments.of("kraft+kip848", "consumer") + )) } // In Scala 2.12, it is necessary to disambiguate the java.util.stream.Stream.of() method call @@ -138,10 +140,19 @@ object BaseConsumerTest { // * KRaft and the classic group protocol // * KRaft with the new group coordinator enabled and the classic group protocol def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly() : java.util.stream.Stream[Arguments] = { - java.util.stream.Stream.of( + util.Arrays.stream(Array( Arguments.of("zk", "classic"), Arguments.of("kraft", "classic"), - Arguments.of("kraft+kip848", "classic")) + Arguments.of("kraft+kip848", "classic") + )) + } + + // For tests that only work with the consumer group protocol, we want to test the following combination: + // * KRaft with the new group coordinator enabled and the consumer group protocol + def getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly(): java.util.stream.Stream[Arguments] = { + util.Arrays.stream(Array( + Arguments.of("kraft+kip848", "consumer") + )) } val updateProducerCount = new AtomicInteger() diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 54be333762..e084454f5f 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -45,7 +45,7 @@ import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{ConsumerGroupState, ElectionType, TopicCollection, TopicPartition, TopicPartitionInfo, TopicPartitionReplica, Uuid} import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT import org.apache.kafka.server.config.Defaults -import org.apache.kafka.storage.internals.log.LogConfig +import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -447,7 +447,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { configs.get(brokerResource2).entries.size) assertEquals(brokers(2).config.brokerId.toString, configs.get(brokerResource2).get(KafkaConfig.BrokerIdProp).value) assertEquals(brokers(2).config.logCleanerThreads.toString, - configs.get(brokerResource2).get(KafkaConfig.LogCleanerThreadsProp).value) + configs.get(brokerResource2).get(CleanerConfig.LOG_CLEANER_THREADS_PROP).value) checkValidAlterConfigs(client, this, topicResource1, topicResource2) } @@ -2532,7 +2532,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { .all().get(15, TimeUnit.SECONDS) val newLogCleanerDeleteRetention = new Properties - newLogCleanerDeleteRetention.put(KafkaConfig.LogCleanerDeleteRetentionMsProp, "34") + newLogCleanerDeleteRetention.put(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, "34") TestUtils.incrementalAlterConfigs(brokers, client, newLogCleanerDeleteRetention, perBrokerConfig = true) .all().get(15, TimeUnit.SECONDS) @@ -2543,14 +2543,14 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { controllerServer.config.nodeId.toString) controllerServer.controller.incrementalAlterConfigs(ANONYMOUS_CONTEXT, Collections.singletonMap(controllerNodeResource, - Collections.singletonMap(KafkaConfig.LogCleanerDeleteRetentionMsProp, + Collections.singletonMap(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, new SimpleImmutableEntry(AlterConfigOp.OpType.SET, "34"))), false).get() ensureConsistentKRaftMetadata() } waitUntilTrue(() => brokers.forall(_.config.originals.getOrDefault( - KafkaConfig.LogCleanerDeleteRetentionMsProp, "").toString.equals("34")), - s"Timed out waiting for change to ${KafkaConfig.LogCleanerDeleteRetentionMsProp}", + CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, "").toString.equals("34")), + s"Timed out waiting for change to ${CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP}", waitTimeMs = 60000L) waitUntilTrue(() => brokers.forall(_.config.originals.getOrDefault( diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerAssignTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerAssignTest.scala new file mode 100644 index 0000000000..219ed9c2a1 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerAssignTest.scala @@ -0,0 +1,246 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR 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.{TestInfoUtils, TestUtils} +import java.util.Properties +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.TopicPartition +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.Timeout +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.{Arguments, MethodSource} + +import org.apache.kafka.common.PartitionInfo +import java.util.stream.Stream +import scala.jdk.CollectionConverters._ +import scala.collection.mutable +import org.junit.jupiter.params.provider.CsvSource + +/** + * Integration tests for the consumer that covers logic related to manual assignment. + */ +@Timeout(600) +class PlaintextConsumerAssignTest extends AbstractConsumerTest { + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAssignAndCommitAsyncNotCommitted(quorum: String, groupProtocol: String): Unit = { + val props = new Properties() + val consumer = createConsumer(configOverrides = props) + val producer = createProducer() + val numRecords = 10000 + val startingTimestamp = System.currentTimeMillis() + val cb = new CountConsumerCommitCallback + sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) + consumer.assign(List(tp).asJava) + consumer.commitAsync(cb) + TestUtils.pollUntilTrue(consumer, () => cb.successCount >= 1 || cb.lastError.isDefined, + "Failed to observe commit callback before timeout", waitTimeMs = 10000) + val committedOffset = consumer.committed(Set(tp).asJava) + assertNotNull(committedOffset) + // No valid fetch position due to the absence of consumer.poll; and therefore no offset was committed to + // tp. The committed offset should be null. This is intentional. + assertNull(committedOffset.get(tp)) + assertTrue(consumer.assignment.contains(tp)) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAssignAndCommitSyncNotCommitted(quorum: String, groupProtocol: String): Unit = { + val props = new Properties() + val consumer = createConsumer(configOverrides = props) + val producer = createProducer() + val numRecords = 10000 + val startingTimestamp = System.currentTimeMillis() + sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) + consumer.assign(List(tp).asJava) + consumer.commitSync() + val committedOffset = consumer.committed(Set(tp).asJava) + assertNotNull(committedOffset) + // No valid fetch position due to the absence of consumer.poll; and therefore no offset was committed to + // tp. The committed offset should be null. This is intentional. + assertNull(committedOffset.get(tp)) + assertTrue(consumer.assignment.contains(tp)) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAssignAndCommitSyncAllConsumed(quorum: String, groupProtocol: String): Unit = { + val numRecords = 10000 + + val producer = createProducer() + val startingTimestamp = System.currentTimeMillis() + sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) + + val props = new Properties() + val consumer = createConsumer(configOverrides = props) + consumer.assign(List(tp).asJava) + consumer.seek(tp, 0) + consumeAndVerifyRecords(consumer = consumer, numRecords, startingOffset = 0, startingTimestamp = startingTimestamp) + + consumer.commitSync() + val committedOffset = consumer.committed(Set(tp).asJava) + assertNotNull(committedOffset) + assertNotNull(committedOffset.get(tp)) + assertEquals(numRecords, committedOffset.get(tp).offset()) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAssignAndConsume(quorum: String, groupProtocol: String): Unit = { + val numRecords = 10 + + val producer = createProducer() + val startingTimestamp = System.currentTimeMillis() + sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) + + val props = new Properties() + val consumer = createConsumer(configOverrides = props, + configsToRemove = List(ConsumerConfig.GROUP_ID_CONFIG)) + consumer.assign(List(tp).asJava) + consumeAndVerifyRecords(consumer = consumer, numRecords, startingOffset = 0, startingTimestamp = startingTimestamp) + + assertEquals(numRecords, consumer.position(tp)) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAssignAndConsumeSkippingPosition(quorum: String, groupProtocol: String): Unit = { + val numRecords = 10 + + val producer = createProducer() + val startingTimestamp = System.currentTimeMillis() + sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) + + val props = new Properties() + val consumer = createConsumer(configOverrides = props, + configsToRemove = List(ConsumerConfig.GROUP_ID_CONFIG)) + consumer.assign(List(tp).asJava) + val offset = 1 + consumer.seek(tp, offset) + consumeAndVerifyRecords(consumer = consumer, numRecords - offset, startingOffset = offset, + startingKeyAndValueIndex = offset, startingTimestamp = startingTimestamp + offset) + + assertEquals(numRecords, consumer.position(tp)) + } + + // partitionsFor not implemented in consumer group protocol and this test requires ZK also + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @CsvSource(Array( + "zk, classic" + )) + def testAssignAndConsumeWithLeaderChangeValidatingPositions(quorum: String, groupProtocol: String): Unit = { + val numRecords = 10 + val producer = createProducer() + val startingTimestamp = System.currentTimeMillis() + sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) + val props = new Properties() + val consumer = createConsumer(configOverrides = props, + configsToRemove = List(ConsumerConfig.GROUP_ID_CONFIG)) + consumer.assign(List(tp).asJava) + consumeAndVerifyRecords(consumer = consumer, numRecords, startingOffset = 0, startingTimestamp = startingTimestamp) + + // Force leader epoch change to trigger position validation + var parts: mutable.Buffer[PartitionInfo] = null + while (parts == null) + parts = consumer.partitionsFor(tp.topic()).asScala + val leader = parts.head.leader().id() + this.servers(leader).shutdown() + this.servers(leader).startup() + + // Consume after leader change + sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) + consumeAndVerifyRecords(consumer = consumer, numRecords, startingOffset = 10, + startingTimestamp = startingTimestamp) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAssignAndFetchCommittedOffsets(quorum: String, groupProtocol: String): Unit = { + val numRecords = 100 + val startingTimestamp = System.currentTimeMillis() + val producer = createProducer() + sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) + val props = new Properties() + val consumer = createConsumer(configOverrides = props) + consumer.assign(List(tp).asJava) + // First consumer consumes and commits offsets + consumer.seek(tp, 0) + consumeAndVerifyRecords(consumer = consumer, numRecords, startingOffset = 0, + startingTimestamp = startingTimestamp) + consumer.commitSync() + assertEquals(numRecords, consumer.committed(Set(tp).asJava).get(tp).offset) + // We should see the committed offsets from another consumer + val anotherConsumer = createConsumer(configOverrides = props) + anotherConsumer.assign(List(tp).asJava) + assertEquals(numRecords, anotherConsumer.committed(Set(tp).asJava).get(tp).offset) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAssignAndConsumeFromCommittedOffsets(quorum: String, groupProtocol: String): Unit = { + val producer = createProducer() + val numRecords = 100 + val startingTimestamp = System.currentTimeMillis() + sendRecords(producer, numRecords = numRecords, tp, startingTimestamp = startingTimestamp) + + // Commit offset with first consumer + val props = new Properties() + props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "group1") + val consumer = createConsumer(configOverrides = props) + consumer.assign(List(tp).asJava) + val offset = 10 + consumer.commitSync(Map[TopicPartition, OffsetAndMetadata]((tp, new OffsetAndMetadata(offset))) + .asJava) + assertEquals(offset, consumer.committed(Set(tp).asJava).get(tp).offset) + consumer.close() + + // Consume from committed offsets with another consumer in same group + val anotherConsumer = createConsumer(configOverrides = props) + assertEquals(offset, anotherConsumer.committed(Set(tp).asJava).get(tp).offset) + anotherConsumer.assign(List(tp).asJava) + consumeAndVerifyRecords(consumer = anotherConsumer, numRecords - offset, + startingOffset = offset, startingKeyAndValueIndex = offset, + startingTimestamp = startingTimestamp + offset) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAssignAndRetrievingCommittedOffsetsMultipleTimes(quorum: String, groupProtocol: String): Unit = { + val numRecords = 100 + val startingTimestamp = System.currentTimeMillis() + val producer = createProducer() + sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) + + val props = new Properties() + val consumer = createConsumer(configOverrides = props) + consumer.assign(List(tp).asJava) + + // Consume and commit offsets + consumer.seek(tp, 0) + consumeAndVerifyRecords(consumer = consumer, numRecords, startingOffset = 0, + startingTimestamp = startingTimestamp) + consumer.commitSync() + + // Check committed offsets twice with same consumer + assertEquals(numRecords, consumer.committed(Set(tp).asJava).get(tp).offset) + assertEquals(numRecords, consumer.committed(Set(tp).asJava).get(tp).offset) + } + +} + +object PlaintextConsumerAssignTest { + def getTestQuorumAndGroupProtocolParametersAll: Stream[Arguments] = + BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll() +} diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerAssignorsTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerAssignorsTest.scala new file mode 100644 index 0000000000..e126774a13 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerAssignorsTest.scala @@ -0,0 +1,403 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package kafka.api + +import kafka.utils.{TestInfoUtils, TestUtils} +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.errors.UnsupportedAssignorException +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.Timeout +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.{Arguments, CsvSource, MethodSource} + +import java.util +import java.util.concurrent.TimeUnit +import java.util.concurrent.locks.ReentrantLock +import java.util.stream.Stream +import scala.collection.mutable.Buffer +import scala.jdk.CollectionConverters._ + +/** + * Integration tests for the consumer that covers assignors logic (client and server side assignors) + */ +@Timeout(600) +class PlaintextConsumerAssignorsTest extends AbstractConsumerTest { + + // Only the classic group protocol supports client-side assignors + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + def testRoundRobinAssignment(quorum: String, groupProtocol: String): Unit = { + // 1 consumer using round-robin assignment + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "roundrobin-group") + this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[RoundRobinAssignor].getName) + val consumer = createConsumer() + + // create two new topics, each having 2 partitions + val topic1 = "topic1" + val topic2 = "topic2" + val producer = createProducer() + val expectedAssignment = createTopicAndSendRecords(producer, topic1, 2, 100) ++ + createTopicAndSendRecords(producer, topic2, 2, 100) + + assertEquals(0, consumer.assignment().size) + + // subscribe to two topics + consumer.subscribe(List(topic1, topic2).asJava) + awaitAssignment(consumer, expectedAssignment) + + // add one more topic with 2 partitions + val topic3 = "topic3" + createTopicAndSendRecords(producer, topic3, 2, 100) + + val newExpectedAssignment = expectedAssignment ++ Set(new TopicPartition(topic3, 0), new TopicPartition(topic3, 1)) + consumer.subscribe(List(topic1, topic2, topic3).asJava) + awaitAssignment(consumer, newExpectedAssignment) + + // remove the topic we just added + consumer.subscribe(List(topic1, topic2).asJava) + awaitAssignment(consumer, expectedAssignment) + + consumer.unsubscribe() + assertEquals(0, consumer.assignment().size) + } + + // Only the classic group protocol supports client-side assignors + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + def testMultiConsumerRoundRobinAssignor(quorum: String, groupProtocol: String): Unit = { + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "roundrobin-group") + this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[RoundRobinAssignor].getName) + + // create two new topics, total number of partitions must be greater than number of consumers + val topic1 = "topic1" + val topic2 = "topic2" + val producer = createProducer() + val subscriptions = createTopicAndSendRecords(producer, topic1, 5, 100) ++ + createTopicAndSendRecords(producer, topic2, 8, 100) + + // create a group of consumers, subscribe the consumers to all the topics and start polling + // for the topic partition assignment + val (consumerGroup, consumerPollers) = createConsumerGroupAndWaitForAssignment(10, List(topic1, topic2), subscriptions) + try { + validateGroupAssignment(consumerPollers, subscriptions) + + // add one more consumer and validate re-assignment + addConsumersToGroupAndWaitForGroupAssignment(1, consumerGroup, consumerPollers, + List(topic1, topic2), subscriptions, "roundrobin-group") + } finally { + consumerPollers.foreach(_.shutdown()) + } + } + + /** + * This test runs the following scenario to verify sticky assignor behavior. + * Topics: single-topic, with random number of partitions, where #par is 10, 20, 30, 40, 50, 60, 70, 80, 90, or 100 + * Consumers: 9 consumers subscribed to the single topic + * Expected initial assignment: partitions are assigned to consumers in a round robin fashion. + * - (#par mod 9) consumers will get (#par / 9 + 1) partitions, and the rest get (#par / 9) partitions + * Then consumer #10 is added to the list (subscribing to the same single topic) + * Expected new assignment: + * - (#par / 10) partition per consumer, where one partition from each of the early (#par mod 9) consumers + * will move to consumer #10, leading to a total of (#par mod 9) partition movement + */ + // Only the classic group protocol supports client-side assignors + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + def testMultiConsumerStickyAssignor(quorum: String, groupProtocol: String): Unit = { + + def reverse(m: Map[Long, Set[TopicPartition]]) = + m.values.toSet.flatten.map(v => (v, m.keys.filter(m(_).contains(v)).head)).toMap + + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "sticky-group") + this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[StickyAssignor].getName) + + // create one new topic + val topic = "single-topic" + val rand = 1 + scala.util.Random.nextInt(10) + val producer = createProducer() + val partitions = createTopicAndSendRecords(producer, topic, rand * 10, 100) + + // create a group of consumers, subscribe the consumers to the single topic and start polling + // for the topic partition assignment + val (consumerGroup, consumerPollers) = createConsumerGroupAndWaitForAssignment(9, List(topic), partitions) + validateGroupAssignment(consumerPollers, partitions) + val prePartition2PollerId = reverse(consumerPollers.map(poller => (poller.getId, poller.consumerAssignment())).toMap) + + // add one more consumer and validate re-assignment + addConsumersToGroupAndWaitForGroupAssignment(1, consumerGroup, consumerPollers, List(topic), partitions, "sticky-group") + + val postPartition2PollerId = reverse(consumerPollers.map(poller => (poller.getId, poller.consumerAssignment())).toMap) + val keys = prePartition2PollerId.keySet.union(postPartition2PollerId.keySet) + var changes = 0 + keys.foreach { key => + val preVal = prePartition2PollerId.get(key) + val postVal = postPartition2PollerId.get(key) + if (preVal.nonEmpty && postVal.nonEmpty) { + if (preVal.get != postVal.get) + changes += 1 + } else + changes += 1 + } + + consumerPollers.foreach(_.shutdown()) + + assertEquals(rand, changes, "Expected only two topic partitions that have switched to other consumers.") + } + + /** + * This test re-uses BaseConsumerTest's consumers. + * As a result, it is testing the default assignment strategy set by BaseConsumerTest + * It tests the assignment results is expected using default assignor (i.e. Range assignor) + */ + // Only the classic group protocol supports client-side assignors + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + def testMultiConsumerDefaultAssignorAndVerifyAssignment(quorum: String, groupProtocol: String): Unit = { + // create two new topics, each having 3 partitions + val topic1 = "topic1" + val topic2 = "topic2" + + createTopic(topic1, 3) + createTopic(topic2, 3) + + val consumersInGroup = Buffer[Consumer[Array[Byte], Array[Byte]]]() + consumersInGroup += createConsumer() + consumersInGroup += createConsumer() + + val tp1_0 = new TopicPartition(topic1, 0) + val tp1_1 = new TopicPartition(topic1, 1) + val tp1_2 = new TopicPartition(topic1, 2) + val tp2_0 = new TopicPartition(topic2, 0) + val tp2_1 = new TopicPartition(topic2, 1) + val tp2_2 = new TopicPartition(topic2, 2) + + val subscriptions = Set(tp1_0, tp1_1, tp1_2, tp2_0, tp2_1, tp2_2) + val consumerPollers = subscribeConsumers(consumersInGroup, List(topic1, topic2)) + + val expectedAssignment = Buffer(Set(tp1_0, tp1_1, tp2_0, tp2_1), Set(tp1_2, tp2_2)) + + try { + validateGroupAssignment(consumerPollers, subscriptions, expectedAssignment = expectedAssignment) + } finally { + consumerPollers.foreach(_.shutdown()) + } + } + + /** + * This test re-uses BaseConsumerTest's consumers. + * As a result, it is testing the default assignment strategy set by BaseConsumerTest + */ + // Only the classic group protocol supports client-side assignors + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + def testMultiConsumerDefaultAssignor(quorum: String, groupProtocol: String): Unit = { + // use consumers and topics defined in this class + one more topic + val producer = createProducer() + sendRecords(producer, numRecords = 100, tp) + sendRecords(producer, numRecords = 100, tp2) + val topic1 = "topic1" + val subscriptions = Set(tp, tp2) ++ createTopicAndSendRecords(producer, topic1, 5, 100) + + // subscribe all consumers to all topics and validate the assignment + + val consumersInGroup = Buffer[Consumer[Array[Byte], Array[Byte]]]() + consumersInGroup += createConsumer() + consumersInGroup += createConsumer() + + val consumerPollers = subscribeConsumers(consumersInGroup, List(topic, topic1)) + try { + validateGroupAssignment(consumerPollers, subscriptions) + + // add 2 more consumers and validate re-assignment + addConsumersToGroupAndWaitForGroupAssignment(2, consumersInGroup, consumerPollers, List(topic, topic1), subscriptions) + + // add one more topic and validate partition re-assignment + val topic2 = "topic2" + val expandedSubscriptions = subscriptions ++ createTopicAndSendRecords(producer, topic2, 3, 100) + changeConsumerGroupSubscriptionAndValidateAssignment(consumerPollers, List(topic, topic1, topic2), expandedSubscriptions) + + // remove the topic we just added and validate re-assignment + changeConsumerGroupSubscriptionAndValidateAssignment(consumerPollers, List(topic, topic1), subscriptions) + + } finally { + consumerPollers.foreach(_.shutdown()) + } + } + + // Remote assignors only supported with consumer group protocol + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @CsvSource(Array( + "kraft+kip848, consumer" + )) + def testRemoteAssignorInvalid(quorum: String, groupProtocol: String): Unit = { + // 1 consumer using invalid remote assignor + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "invalid-assignor-group") + this.consumerConfig.setProperty(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "invalid") + val consumer = createConsumer() + + // create two new topics, each having 2 partitions + val topic1 = "topic1" + val producer = createProducer() + val expectedAssignment = createTopicAndSendRecords(producer, topic1, 2, 100) + + assertEquals(0, consumer.assignment().size) + + // subscribe to two topics + consumer.subscribe(List(topic1).asJava) + + val e: UnsupportedAssignorException = assertThrows( + classOf[UnsupportedAssignorException], + () => awaitAssignment(consumer, expectedAssignment) + ) + + assertTrue(e.getMessage.startsWith("ServerAssignor invalid is not supported. " + + "Supported assignors: ")) + } + + // Remote assignors only supported with consumer group protocol + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @CsvSource(Array( + "kraft+kip848, consumer" + )) + def testRemoteAssignorRange(quorum: String, groupProtocol: String): Unit = { + // 1 consumer using range assignment + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "range-group") + this.consumerConfig.setProperty(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "range") + this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, "30000") + val consumer = createConsumer() + + // create two new topics, each having 2 partitions + val topic1 = "topic1" + val topic2 = "topic2" + val producer = createProducer() + val expectedAssignment = createTopicAndSendRecords(producer, topic1, 2, 100) ++ + createTopicAndSendRecords(producer, topic2, 2, 100) + + assertEquals(0, consumer.assignment().size) + + // subscribe to two topics + consumer.subscribe(List(topic1, topic2).asJava) + awaitAssignment(consumer, expectedAssignment) + + // add one more topic with 2 partitions + val topic3 = "topic3" + val additionalAssignment = createTopicAndSendRecords(producer, topic3, 2, 100) + + val newExpectedAssignment = expectedAssignment ++ additionalAssignment + consumer.subscribe(List(topic1, topic2, topic3).asJava) + awaitAssignment(consumer, newExpectedAssignment) + + // remove the topic we just added + consumer.subscribe(List(topic1, topic2).asJava) + awaitAssignment(consumer, expectedAssignment) + + consumer.unsubscribe() + assertEquals(0, consumer.assignment().size) + } + + // Only the classic group protocol supports client-side assignors + @ParameterizedTest + @CsvSource(Array( + "org.apache.kafka.clients.consumer.CooperativeStickyAssignor, zk", + "org.apache.kafka.clients.consumer.RangeAssignor, zk", + "org.apache.kafka.clients.consumer.CooperativeStickyAssignor, kraft", + "org.apache.kafka.clients.consumer.RangeAssignor, kraft" + )) + def testRebalanceAndRejoin(assignmentStrategy: String, quorum: String): Unit = { + // create 2 consumers + this.consumerConfig.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, "classic") + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "rebalance-and-rejoin-group") + this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, assignmentStrategy) + this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true") + val consumer1 = createConsumer() + val consumer2 = createConsumer() + + // create a new topic, have 2 partitions + val topic = "topic1" + val producer = createProducer() + val expectedAssignment = createTopicAndSendRecords(producer, topic, 2, 100) + + assertEquals(0, consumer1.assignment().size) + assertEquals(0, consumer2.assignment().size) + + val lock = new ReentrantLock() + var generationId1 = -1 + var memberId1 = "" + val customRebalanceListener = new ConsumerRebalanceListener { + override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = { + } + + override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]): Unit = { + if (!lock.tryLock(3000, TimeUnit.MILLISECONDS)) { + fail(s"Time out while awaiting for lock.") + } + try { + generationId1 = consumer1.groupMetadata().generationId() + memberId1 = consumer1.groupMetadata().memberId() + } finally { + lock.unlock() + } + } + } + val consumerPoller1 = new ConsumerAssignmentPoller(consumer1, List(topic), Set.empty, customRebalanceListener) + consumerPoller1.start() + TestUtils.waitUntilTrue(() => consumerPoller1.consumerAssignment() == expectedAssignment, + s"Timed out while awaiting expected assignment change to $expectedAssignment.") + + // Since the consumer1 already completed the rebalance, + // the `onPartitionsAssigned` rebalance listener will be invoked to set the generationId and memberId + var stableGeneration = -1 + var stableMemberId1 = "" + if (!lock.tryLock(3000, TimeUnit.MILLISECONDS)) { + fail(s"Time out while awaiting for lock.") + } + try { + stableGeneration = generationId1 + stableMemberId1 = memberId1 + } finally { + lock.unlock() + } + + val consumerPoller2 = subscribeConsumerAndStartPolling(consumer2, List(topic)) + TestUtils.waitUntilTrue(() => consumerPoller1.consumerAssignment().size == 1, + s"Timed out while awaiting expected assignment size change to 1.") + TestUtils.waitUntilTrue(() => consumerPoller2.consumerAssignment().size == 1, + s"Timed out while awaiting expected assignment size change to 1.") + + if (!lock.tryLock(3000, TimeUnit.MILLISECONDS)) { + fail(s"Time out while awaiting for lock.") + } + try { + if (assignmentStrategy.equals(classOf[CooperativeStickyAssignor].getName)) { + // cooperative rebalance should rebalance twice before finally stable + assertEquals(stableGeneration + 2, generationId1) + } else { + // eager rebalance should rebalance once before finally stable + assertEquals(stableGeneration + 1, generationId1) + } + assertEquals(stableMemberId1, memberId1) + } finally { + lock.unlock() + } + + consumerPoller1.shutdown() + consumerPoller2.shutdown() + } + +} + +object PlaintextConsumerAssignorsTest { + def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly: Stream[Arguments] = + BaseConsumerTest.getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly() +} diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerFetchTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerFetchTest.scala new file mode 100644 index 0000000000..3012a6d25f --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerFetchTest.scala @@ -0,0 +1,243 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package kafka.api + +import kafka.utils.TestInfoUtils +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.clients.producer.ProducerRecord +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.Timeout +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.{Arguments, MethodSource} +import org.apache.kafka.common.TopicPartition + +import java.time.Duration +import java.util.stream.Stream +import scala.jdk.CollectionConverters._ + +/** + * Integration tests for the consumer that covers fetching logic + */ +@Timeout(600) +class PlaintextConsumerFetchTest extends AbstractConsumerTest { + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testFetchInvalidOffset(quorum: String, groupProtocol: String): Unit = { + this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + val consumer = createConsumer(configOverrides = this.consumerConfig) + + // produce one record + val totalRecords = 2 + val producer = createProducer() + sendRecords(producer, totalRecords, tp) + consumer.assign(List(tp).asJava) + + // poll should fail because there is no offset reset strategy set. + // we fail only when resetting positions after coordinator is known, so using a long timeout. + assertThrows(classOf[NoOffsetForPartitionException], () => consumer.poll(Duration.ofMillis(15000))) + + // seek to out of range position + val outOfRangePos = totalRecords + 1 + consumer.seek(tp, outOfRangePos) + val e = assertThrows(classOf[OffsetOutOfRangeException], () => consumer.poll(Duration.ofMillis(20000))) + val outOfRangePartitions = e.offsetOutOfRangePartitions() + assertNotNull(outOfRangePartitions) + assertEquals(1, outOfRangePartitions.size) + assertEquals(outOfRangePos.toLong, outOfRangePartitions.get(tp)) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testFetchOutOfRangeOffsetResetConfigEarliest(quorum: String, groupProtocol: String): Unit = { + this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + // ensure no in-flight fetch request so that the offset can be reset immediately + this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG, "0") + val consumer = createConsumer(configOverrides = this.consumerConfig) + val totalRecords = 10L + + val producer = createProducer() + val startingTimestamp = 0 + sendRecords(producer, totalRecords.toInt, tp, startingTimestamp = startingTimestamp) + consumer.assign(List(tp).asJava) + consumeAndVerifyRecords(consumer = consumer, numRecords = totalRecords.toInt, startingOffset = 0) + // seek to out of range position + val outOfRangePos = totalRecords + 1 + consumer.seek(tp, outOfRangePos) + // assert that poll resets to the beginning position + consumeAndVerifyRecords(consumer = consumer, numRecords = 1, startingOffset = 0) + } + + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testFetchOutOfRangeOffsetResetConfigLatest(quorum: String, groupProtocol: String): Unit = { + this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest") + // ensure no in-flight fetch request so that the offset can be reset immediately + this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG, "0") + val consumer = createConsumer(configOverrides = this.consumerConfig) + val totalRecords = 10L + + val producer = createProducer() + val startingTimestamp = 0 + sendRecords(producer, totalRecords.toInt, tp, startingTimestamp = startingTimestamp) + consumer.assign(List(tp).asJava) + consumer.seek(tp, 0) + // consume some, but not all of the records + consumeAndVerifyRecords(consumer = consumer, numRecords = totalRecords.toInt / 2, startingOffset = 0) + // seek to out of range position + val outOfRangePos = totalRecords + 17 // arbitrary, much higher offset + consumer.seek(tp, outOfRangePos) + // assert that poll resets to the ending position + assertTrue(consumer.poll(Duration.ofMillis(50)).isEmpty) + sendRecords(producer, totalRecords.toInt, tp, startingTimestamp = totalRecords) + val nextRecord = consumer.poll(Duration.ofMillis(50)).iterator().next() + // ensure the seek went to the last known record at the time of the previous poll + assertEquals(totalRecords, nextRecord.offset()) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testFetchRecordLargerThanFetchMaxBytes(quorum: String, groupProtocol: String): Unit = { + val maxFetchBytes = 10 * 1024 + this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, maxFetchBytes.toString) + checkLargeRecord(maxFetchBytes + 1) + } + + private def checkLargeRecord(producerRecordSize: Int): Unit = { + val consumer = createConsumer() + + // produce a record that is larger than the configured fetch size + val record = new ProducerRecord(tp.topic(), tp.partition(), "key".getBytes, + new Array[Byte](producerRecordSize)) + val producer = createProducer() + producer.send(record) + + // consuming a record that is too large should succeed since KIP-74 + consumer.assign(List(tp).asJava) + val records = consumer.poll(Duration.ofMillis(20000)) + assertEquals(1, records.count) + val consumerRecord = records.iterator().next() + assertEquals(0L, consumerRecord.offset) + assertEquals(tp.topic(), consumerRecord.topic()) + assertEquals(tp.partition(), consumerRecord.partition()) + assertArrayEquals(record.key(), consumerRecord.key()) + assertArrayEquals(record.value(), consumerRecord.value()) + } + + /** We should only return a large record if it's the first record in the first non-empty partition of the fetch request */ + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testFetchHonoursFetchSizeIfLargeRecordNotFirst(quorum: String, groupProtocol: String): Unit = { + val maxFetchBytes = 10 * 1024 + this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, maxFetchBytes.toString) + checkFetchHonoursSizeIfLargeRecordNotFirst(maxFetchBytes) + } + + private def checkFetchHonoursSizeIfLargeRecordNotFirst(largeProducerRecordSize: Int): Unit = { + val consumer = createConsumer() + + val smallRecord = new ProducerRecord(tp.topic(), tp.partition(), "small".getBytes, + "value".getBytes) + val largeRecord = new ProducerRecord(tp.topic(), tp.partition(), "large".getBytes, + new Array[Byte](largeProducerRecordSize)) + + val producer = createProducer() + producer.send(smallRecord).get + producer.send(largeRecord).get + + // we should only get the small record in the first `poll` + consumer.assign(List(tp).asJava) + val records = consumer.poll(Duration.ofMillis(20000)) + assertEquals(1, records.count) + val consumerRecord = records.iterator().next() + assertEquals(0L, consumerRecord.offset) + assertEquals(tp.topic(), consumerRecord.topic()) + assertEquals(tp.partition(), consumerRecord.partition()) + assertArrayEquals(smallRecord.key(), consumerRecord.key()) + assertArrayEquals(smallRecord.value(), consumerRecord.value()) + } + + /** We should only return a large record if it's the first record in the first partition of the fetch request */ + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testFetchHonoursMaxPartitionFetchBytesIfLargeRecordNotFirst(quorum: String, groupProtocol: String): Unit = { + val maxPartitionFetchBytes = 10 * 1024 + this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, maxPartitionFetchBytes.toString) + checkFetchHonoursSizeIfLargeRecordNotFirst(maxPartitionFetchBytes) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testFetchRecordLargerThanMaxPartitionFetchBytes(quorum: String, groupProtocol: String): Unit = { + val maxPartitionFetchBytes = 10 * 1024 + this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, maxPartitionFetchBytes.toString) + checkLargeRecord(maxPartitionFetchBytes + 1) + } + + /** Test that we consume all partitions if fetch max bytes and max.partition.fetch.bytes are low */ + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testLowMaxFetchSizeForRequestAndPartition(quorum: String, groupProtocol: String): Unit = { + // one of the effects of this is that there will be some log reads where `0 > remaining limit bytes < message size` + // and we don't return the message because it's not the first message in the first non-empty partition of the fetch + // this behaves a little different than when remaining limit bytes is 0 and it's important to test it + this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, "500") + this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, "100") + + // Avoid a rebalance while the records are being sent (the default is 6 seconds) + this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 20000.toString) + val consumer = createConsumer() + + val topic1 = "topic1" + val topic2 = "topic2" + val topic3 = "topic3" + val partitionCount = 30 + val topics = Seq(topic1, topic2, topic3) + topics.foreach { topicName => + createTopic(topicName, partitionCount, brokerCount) + } + + val partitions = topics.flatMap { topic => + (0 until partitionCount).map(new TopicPartition(topic, _)) + } + + assertEquals(0, consumer.assignment().size) + + consumer.subscribe(List(topic1, topic2, topic3).asJava) + + awaitAssignment(consumer, partitions.toSet) + + val producer = createProducer() + + val producerRecords = partitions.flatMap(sendRecords(producer, numRecords = partitionCount, _)) + + val consumerRecords = consumeRecords(consumer, producerRecords.size) + + val expected = producerRecords.map { record => + (record.topic, record.partition, new String(record.key), new String(record.value), record.timestamp) + }.toSet + + val actual = consumerRecords.map { record => + (record.topic, record.partition, new String(record.key), new String(record.value), record.timestamp) + }.toSet + + assertEquals(expected, actual) + } + +} + +object PlaintextConsumerFetchTest { + def getTestQuorumAndGroupProtocolParametersAll: Stream[Arguments] = + BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll() +} diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerPollTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerPollTest.scala new file mode 100644 index 0000000000..3bcc449162 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerPollTest.scala @@ -0,0 +1,284 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR 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.TestInfoUtils +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.{MetricName, TopicPartition} +import org.apache.kafka.common.utils.Utils +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.Timeout +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.{Arguments, MethodSource} + +import java.time.Duration +import java.util +import java.util.stream.Stream +import scala.jdk.CollectionConverters._ +import scala.collection.mutable.Buffer + +/** + * Integration tests for the consumer that covers the poll logic + */ +@Timeout(600) +class PlaintextConsumerPollTest extends AbstractConsumerTest { + + // Deprecated poll(timeout) not supported for consumer group protocol + @deprecated("poll(Duration) is the replacement", since = "2.0") + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + def testDeprecatedPollBlocksForAssignment(quorum: String, groupProtocol: String): Unit = { + val consumer = createConsumer() + consumer.subscribe(Set(topic).asJava) + consumer.poll(0) + assertEquals(Set(tp, tp2), consumer.assignment().asScala) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testMaxPollRecords(quorum: String, groupProtocol: String): Unit = { + val maxPollRecords = 2 + val numRecords = 10000 + + val producer = createProducer() + val startingTimestamp = System.currentTimeMillis() + sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) + + this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString) + val consumer = createConsumer() + consumer.assign(List(tp).asJava) + consumeAndVerifyRecords(consumer, numRecords = numRecords, startingOffset = 0, maxPollRecords = maxPollRecords, + startingTimestamp = startingTimestamp) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testMaxPollIntervalMs(quorum: String, groupProtocol: String): Unit = { + this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 1000.toString) + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString) + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 2000.toString) + + val consumer = createConsumer() + + val listener = new TestConsumerReassignmentListener() + consumer.subscribe(List(topic).asJava, listener) + + // rebalance to get the initial assignment + awaitRebalance(consumer, listener) + assertEquals(1, listener.callsToAssigned) + assertEquals(0, listener.callsToRevoked) + + // after we extend longer than max.poll a rebalance should be triggered + // NOTE we need to have a relatively much larger value than max.poll to let heartbeat expired for sure + Thread.sleep(3000) + + awaitRebalance(consumer, listener) + assertEquals(2, listener.callsToAssigned) + assertEquals(1, listener.callsToRevoked) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testMaxPollIntervalMsDelayInRevocation(quorum: String, groupProtocol: String): Unit = { + this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 5000.toString) + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString) + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000.toString) + this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false.toString) + + val consumer = createConsumer() + var commitCompleted = false + var committedPosition: Long = -1 + + val listener = new TestConsumerReassignmentListener { + override def onPartitionsLost(partitions: util.Collection[TopicPartition]): Unit = {} + + override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = { + if (!partitions.isEmpty && partitions.contains(tp)) { + // on the second rebalance (after we have joined the group initially), sleep longer + // than session timeout and then try a commit. We should still be in the group, + // so the commit should succeed + Utils.sleep(1500) + committedPosition = consumer.position(tp) + consumer.commitSync(Map(tp -> new OffsetAndMetadata(committedPosition)).asJava) + commitCompleted = true + } + super.onPartitionsRevoked(partitions) + } + } + + consumer.subscribe(List(topic).asJava, listener) + + // rebalance to get the initial assignment + awaitRebalance(consumer, listener) + + // force a rebalance to trigger an invocation of the revocation callback while in the group + consumer.subscribe(List("otherTopic").asJava, listener) + awaitRebalance(consumer, listener) + + assertEquals(0, committedPosition) + assertTrue(commitCompleted) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testMaxPollIntervalMsDelayInAssignment(quorum: String, groupProtocol: String): Unit = { + this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 5000.toString) + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString) + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000.toString) + this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false.toString) + + val consumer = createConsumer() + val listener = new TestConsumerReassignmentListener { + override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]): Unit = { + // sleep longer than the session timeout, we should still be in the group after invocation + Utils.sleep(1500) + super.onPartitionsAssigned(partitions) + } + } + consumer.subscribe(List(topic).asJava, listener) + + // rebalance to get the initial assignment + awaitRebalance(consumer, listener) + + // We should still be in the group after this invocation + ensureNoRebalance(consumer, listener) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testMaxPollIntervalMsShorterThanPollTimeout(quorum: String, groupProtocol: String): Unit = { + this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 1000.toString) + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString) + + val consumer = createConsumer() + val listener = new TestConsumerReassignmentListener + consumer.subscribe(List(topic).asJava, listener) + + // rebalance to get the initial assignment + awaitRebalance(consumer, listener) + + val callsToAssignedAfterFirstRebalance = listener.callsToAssigned + + consumer.poll(Duration.ofMillis(2000)) + + // If the poll poll above times out, it would trigger a rebalance. + // Leave some time for the rebalance to happen and check for the rebalance event. + consumer.poll(Duration.ofMillis(500)) + consumer.poll(Duration.ofMillis(500)) + + assertEquals(callsToAssignedAfterFirstRebalance, listener.callsToAssigned) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testPerPartitionLeadWithMaxPollRecords(quorum: String, groupProtocol: String): Unit = { + val numMessages = 1000 + val maxPollRecords = 10 + val producer = createProducer() + sendRecords(producer, numMessages, tp) + + consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testPerPartitionLeadWithMaxPollRecords") + consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLeadWithMaxPollRecords") + consumerConfig.setProperty(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString) + val consumer = createConsumer() + consumer.assign(List(tp).asJava) + awaitNonEmptyRecords(consumer, tp) + + val tags = new util.HashMap[String, String]() + tags.put("client-id", "testPerPartitionLeadWithMaxPollRecords") + tags.put("topic", tp.topic()) + tags.put("partition", String.valueOf(tp.partition())) + val lead = consumer.metrics.get(new MetricName("records-lead", "consumer-fetch-manager-metrics", "", tags)) + assertEquals(maxPollRecords, lead.metricValue().asInstanceOf[Double], s"The lead should be $maxPollRecords") + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testPerPartitionLagWithMaxPollRecords(quorum: String, groupProtocol: String): Unit = { + val numMessages = 1000 + val maxPollRecords = 10 + val producer = createProducer() + sendRecords(producer, numMessages, tp) + + consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testPerPartitionLagWithMaxPollRecords") + consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLagWithMaxPollRecords") + consumerConfig.setProperty(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString) + val consumer = createConsumer() + consumer.assign(List(tp).asJava) + val records = awaitNonEmptyRecords(consumer, tp) + + 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(numMessages - records.count, lag.metricValue.asInstanceOf[Double], epsilon, s"The lag should be ${numMessages - records.count}") + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testMultiConsumerSessionTimeoutOnStopPolling(quorum: String, groupProtocol: String): Unit = { + runMultiConsumerSessionTimeoutTest(false) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testMultiConsumerSessionTimeoutOnClose(quorum: String, groupProtocol: String): Unit = { + runMultiConsumerSessionTimeoutTest(true) + } + + def runMultiConsumerSessionTimeoutTest(closeConsumer: Boolean): Unit = { + // use consumers defined in this class plus one additional consumer + // Use topic defined in this class + one additional topic + val producer = createProducer() + sendRecords(producer, numRecords = 100, tp) + sendRecords(producer, numRecords = 100, tp2) + val topic1 = "topic1" + val subscriptions = Set(tp, tp2) ++ createTopicAndSendRecords(producer, topic1, 6, 100) + + // first subscribe consumers that are defined in this class + val consumerPollers = Buffer[ConsumerAssignmentPoller]() + consumerPollers += subscribeConsumerAndStartPolling(createConsumer(), List(topic, topic1)) + consumerPollers += subscribeConsumerAndStartPolling(createConsumer(), List(topic, topic1)) + + // create one more consumer and add it to the group; we will timeout this consumer + val timeoutConsumer = createConsumer() + val timeoutPoller = subscribeConsumerAndStartPolling(timeoutConsumer, List(topic, topic1)) + consumerPollers += timeoutPoller + + // validate the initial assignment + validateGroupAssignment(consumerPollers, subscriptions) + + // stop polling and close one of the consumers, should trigger partition re-assignment among alive consumers + timeoutPoller.shutdown() + consumerPollers -= timeoutPoller + if (closeConsumer) + timeoutConsumer.close() + + validateGroupAssignment(consumerPollers, subscriptions, + Some(s"Did not get valid assignment for partitions ${subscriptions.asJava} after one consumer left"), 3 * groupMaxSessionTimeoutMs) + + // done with pollers and consumers + for (poller <- consumerPollers) + poller.shutdown() + } +} + +object PlaintextConsumerPollTest { + def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly: Stream[Arguments] = + BaseConsumerTest.getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly() + + def getTestQuorumAndGroupProtocolParametersAll: Stream[Arguments] = + BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll() +} diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerSubscriptionTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerSubscriptionTest.scala new file mode 100644 index 0000000000..7ac0501bb5 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerSubscriptionTest.scala @@ -0,0 +1,237 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR 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.TestInfoUtils +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.TopicPartition +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.Timeout +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.{Arguments, MethodSource} + +import java.util.regex.Pattern +import java.util.stream.Stream +import scala.jdk.CollectionConverters._ + +/** + * Integration tests for the consumer that covers the subscribe and unsubscribe logic. + */ +@Timeout(600) +class PlaintextConsumerSubscriptionTest extends AbstractConsumerTest { + + /** + * Verifies that pattern subscription performs as expected. + * The pattern matches the topics 'topic' and 'tblablac', but not 'tblablak' or 'tblab1'. + * It is expected that the consumer is subscribed to all partitions of 'topic' and + * 'tblablac' after the subscription when metadata is refreshed. + * When a new topic 'tsomec' is added afterwards, it is expected that upon the next + * metadata refresh the consumer becomes subscribed to this new topic and all partitions + * of that topic are assigned to it. + */ + // TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented. + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + def testPatternSubscription(quorum: String, groupProtocol: String): Unit = { + val numRecords = 10000 + val producer = createProducer() + sendRecords(producer, numRecords, tp) + + val topic1 = "tblablac" // matches subscribed pattern + createTopic(topic1, 2, brokerCount) + sendRecords(producer, numRecords = 1000, new TopicPartition(topic1, 0)) + sendRecords(producer, numRecords = 1000, new TopicPartition(topic1, 1)) + + val topic2 = "tblablak" // does not match subscribed pattern + createTopic(topic2, 2, brokerCount) + sendRecords(producer, numRecords = 1000, new TopicPartition(topic2, 0)) + sendRecords(producer, numRecords = 1000, new TopicPartition(topic2, 1)) + + val topic3 = "tblab1" // does not match subscribed pattern + createTopic(topic3, 2, brokerCount) + sendRecords(producer, numRecords = 1000, new TopicPartition(topic3, 0)) + sendRecords(producer, numRecords = 1000, new TopicPartition(topic3, 1)) + + val consumer = createConsumer() + assertEquals(0, consumer.assignment().size) + + val pattern = Pattern.compile("t.*c") + consumer.subscribe(pattern, new TestConsumerReassignmentListener) + + var assignment = Set( + new TopicPartition(topic, 0), + new TopicPartition(topic, 1), + new TopicPartition(topic1, 0), + new TopicPartition(topic1, 1)) + awaitAssignment(consumer, assignment) + + val topic4 = "tsomec" // matches subscribed pattern + createTopic(topic4, 2, brokerCount) + sendRecords(producer, numRecords = 1000, new TopicPartition(topic4, 0)) + sendRecords(producer, numRecords = 1000, new TopicPartition(topic4, 1)) + + assignment ++= Set( + new TopicPartition(topic4, 0), + new TopicPartition(topic4, 1)) + awaitAssignment(consumer, assignment) + + consumer.unsubscribe() + assertEquals(0, consumer.assignment().size) + } + + /** + * Verifies that a second call to pattern subscription succeeds and performs as expected. + * The initial subscription is to a pattern that matches two topics 'topic' and 'foo'. + * The second subscription is to a pattern that matches 'foo' and a new topic 'bar'. + * It is expected that the consumer is subscribed to all partitions of 'topic' and 'foo' after + * the first subscription, and to all partitions of 'foo' and 'bar' after the second. + * The metadata refresh interval is intentionally increased to a large enough value to guarantee + * that it is the subscription call that triggers a metadata refresh, and not the timeout. + */ + // TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented. + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + def testSubsequentPatternSubscription(quorum: String, groupProtocol: String): Unit = { + this.consumerConfig.setProperty(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "30000") + val consumer = createConsumer() + + val numRecords = 10000 + val producer = createProducer() + sendRecords(producer, numRecords = numRecords, tp) + + // the first topic ('topic') matches first subscription pattern only + + val fooTopic = "foo" // matches both subscription patterns + createTopic(fooTopic, 1, brokerCount) + sendRecords(producer, numRecords = 1000, new TopicPartition(fooTopic, 0)) + + assertEquals(0, consumer.assignment().size) + + val pattern1 = Pattern.compile(".*o.*") // only 'topic' and 'foo' match this + consumer.subscribe(pattern1, new TestConsumerReassignmentListener) + + var assignment = Set( + new TopicPartition(topic, 0), + new TopicPartition(topic, 1), + new TopicPartition(fooTopic, 0)) + awaitAssignment(consumer, assignment) + + val barTopic = "bar" // matches the next subscription pattern + createTopic(barTopic, 1, brokerCount) + sendRecords(producer, numRecords = 1000, new TopicPartition(barTopic, 0)) + + val pattern2 = Pattern.compile("...") // only 'foo' and 'bar' match this + consumer.subscribe(pattern2, new TestConsumerReassignmentListener) + assignment --= Set( + new TopicPartition(topic, 0), + new TopicPartition(topic, 1)) + assignment ++= Set( + new TopicPartition(barTopic, 0)) + awaitAssignment(consumer, assignment) + + consumer.unsubscribe() + assertEquals(0, consumer.assignment().size) + } + + /** + * Verifies that pattern unsubscription performs as expected. + * The pattern matches the topics 'topic' and 'tblablac'. + * It is expected that the consumer is subscribed to all partitions of 'topic' and + * 'tblablac' after the subscription when metadata is refreshed. + * When consumer unsubscribes from all its subscriptions, it is expected that its + * assignments are cleared right away. + */ + // TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented. + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + def testPatternUnsubscription(quorum: String, groupProtocol: String): Unit = { + val numRecords = 10000 + val producer = createProducer() + sendRecords(producer, numRecords, tp) + + val topic1 = "tblablac" // matches the subscription pattern + createTopic(topic1, 2, brokerCount) + sendRecords(producer, numRecords = 1000, new TopicPartition(topic1, 0)) + sendRecords(producer, numRecords = 1000, new TopicPartition(topic1, 1)) + + val consumer = createConsumer() + assertEquals(0, consumer.assignment().size) + + consumer.subscribe(Pattern.compile("t.*c"), new TestConsumerReassignmentListener) + val assignment = Set( + new TopicPartition(topic, 0), + new TopicPartition(topic, 1), + new TopicPartition(topic1, 0), + new TopicPartition(topic1, 1)) + awaitAssignment(consumer, assignment) + + consumer.unsubscribe() + assertEquals(0, consumer.assignment().size) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testExpandingTopicSubscriptions(quorum: String, groupProtocol: String): Unit = { + val otherTopic = "other" + val initialAssignment = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) + val consumer = createConsumer() + consumer.subscribe(List(topic).asJava) + awaitAssignment(consumer, initialAssignment) + + createTopic(otherTopic, 2, brokerCount) + val expandedAssignment = initialAssignment ++ Set(new TopicPartition(otherTopic, 0), new TopicPartition(otherTopic, 1)) + consumer.subscribe(List(topic, otherTopic).asJava) + awaitAssignment(consumer, expandedAssignment) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testShrinkingTopicSubscriptions(quorum: String, groupProtocol: String): Unit = { + val otherTopic = "other" + createTopic(otherTopic, 2, brokerCount) + val initialAssignment = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1), new TopicPartition(otherTopic, 0), new TopicPartition(otherTopic, 1)) + val consumer = createConsumer() + consumer.subscribe(List(topic, otherTopic).asJava) + awaitAssignment(consumer, initialAssignment) + + val shrunkenAssignment = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) + consumer.subscribe(List(topic).asJava) + awaitAssignment(consumer, shrunkenAssignment) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testUnsubscribeTopic(quorum: String, groupProtocol: String): Unit = { + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100") // timeout quickly to avoid slow test + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30") + val consumer = createConsumer() + + val listener = new TestConsumerReassignmentListener() + consumer.subscribe(List(topic).asJava, listener) + + // the initial subscription should cause a callback execution + awaitRebalance(consumer, listener) + + consumer.subscribe(List[String]().asJava) + assertEquals(0, consumer.assignment.size()) + } + +} + +object PlaintextConsumerSubscriptionTest { + def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly: Stream[Arguments] = + BaseConsumerTest.getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly() + + def getTestQuorumAndGroupProtocolParametersAll: Stream[Arguments] = + BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll() +} diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 01dbb206a0..8784e5b43f 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -15,30 +15,24 @@ package kafka.api import java.time.Duration import java.util import java.util.Arrays.asList -import java.util.concurrent.TimeUnit -import java.util.concurrent.locks.ReentrantLock -import java.util.regex.Pattern import java.util.{Locale, Optional, Properties} import kafka.server.{KafkaBroker, QuotaType} import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.admin.{NewPartitions, NewTopic} import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} -import org.apache.kafka.common.{KafkaException, MetricName, PartitionInfo, TopicPartition} +import org.apache.kafka.common.{KafkaException, MetricName, TopicPartition} import org.apache.kafka.common.config.TopicConfig -import org.apache.kafka.common.errors.{InvalidGroupIdException, InvalidTopicException, UnsupportedAssignorException} +import org.apache.kafka.common.errors.{InvalidGroupIdException, InvalidTopicException} import org.apache.kafka.common.header.Headers import org.apache.kafka.common.record.{CompressionType, TimestampType} import org.apache.kafka.common.serialization._ -import org.apache.kafka.common.utils.Utils import org.apache.kafka.test.{MockConsumerInterceptor, MockProducerInterceptor} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Timeout import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{CsvSource, MethodSource} -import scala.collection.mutable -import scala.collection.mutable.Buffer import scala.jdk.CollectionConverters._ @Timeout(600) @@ -131,17 +125,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(numRecords, records.size) } - // Deprecated poll(timeout) not supported for consumer group protocol - @deprecated("poll(Duration) is the replacement", since = "2.0") - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) - def testDeprecatedPollBlocksForAssignment(quorum: String, groupProtocol: String): Unit = { - val consumer = createConsumer() - consumer.subscribe(Set(topic).asJava) - consumer.poll(0) - assertEquals(Set(tp, tp2), consumer.assignment().asScala) - } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testHeadersSerializerDeserializer(quorum: String, groupProtocol: String): Unit = { @@ -152,141 +135,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { testHeadersSerializeDeserialize(extendedSerializer, extendedDeserializer) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testMaxPollRecords(quorum: String, groupProtocol: String): Unit = { - val maxPollRecords = 2 - val numRecords = 10000 - - val producer = createProducer() - val startingTimestamp = System.currentTimeMillis() - sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) - - this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString) - val consumer = createConsumer() - consumer.assign(List(tp).asJava) - consumeAndVerifyRecords(consumer, numRecords = numRecords, startingOffset = 0, maxPollRecords = maxPollRecords, - startingTimestamp = startingTimestamp) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testMaxPollIntervalMs(quorum: String, groupProtocol: String): Unit = { - this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 1000.toString) - this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString) - this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 2000.toString) - - val consumer = createConsumer() - - val listener = new TestConsumerReassignmentListener() - consumer.subscribe(List(topic).asJava, listener) - - // rebalance to get the initial assignment - awaitRebalance(consumer, listener) - assertEquals(1, listener.callsToAssigned) - assertEquals(0, listener.callsToRevoked) - - // after we extend longer than max.poll a rebalance should be triggered - // NOTE we need to have a relatively much larger value than max.poll to let heartbeat expired for sure - Thread.sleep(3000) - - awaitRebalance(consumer, listener) - assertEquals(2, listener.callsToAssigned) - assertEquals(1, listener.callsToRevoked) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testMaxPollIntervalMsDelayInRevocation(quorum: String, groupProtocol: String): Unit = { - this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 5000.toString) - this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString) - this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000.toString) - this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false.toString) - - val consumer = createConsumer() - var commitCompleted = false - var committedPosition: Long = -1 - - val listener = new TestConsumerReassignmentListener { - override def onPartitionsLost(partitions: util.Collection[TopicPartition]): Unit = {} - override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = { - if (!partitions.isEmpty && partitions.contains(tp)) { - // on the second rebalance (after we have joined the group initially), sleep longer - // than session timeout and then try a commit. We should still be in the group, - // so the commit should succeed - Utils.sleep(1500) - committedPosition = consumer.position(tp) - consumer.commitSync(Map(tp -> new OffsetAndMetadata(committedPosition)).asJava) - commitCompleted = true - } - super.onPartitionsRevoked(partitions) - } - } - - consumer.subscribe(List(topic).asJava, listener) - - // rebalance to get the initial assignment - awaitRebalance(consumer, listener) - - // force a rebalance to trigger an invocation of the revocation callback while in the group - consumer.subscribe(List("otherTopic").asJava, listener) - awaitRebalance(consumer, listener) - - assertEquals(0, committedPosition) - assertTrue(commitCompleted) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testMaxPollIntervalMsDelayInAssignment(quorum: String, groupProtocol: String): Unit = { - this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 5000.toString) - this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString) - this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000.toString) - this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false.toString) - - val consumer = createConsumer() - val listener = new TestConsumerReassignmentListener { - override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]): Unit = { - // sleep longer than the session timeout, we should still be in the group after invocation - Utils.sleep(1500) - super.onPartitionsAssigned(partitions) - } - } - consumer.subscribe(List(topic).asJava, listener) - - // rebalance to get the initial assignment - awaitRebalance(consumer, listener) - - // We should still be in the group after this invocation - ensureNoRebalance(consumer, listener) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testMaxPollIntervalMsShorterThanPollTimeout(quorum: String, groupProtocol: String): Unit = { - this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 1000.toString) - this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString) - - val consumer = createConsumer() - val listener = new TestConsumerReassignmentListener - consumer.subscribe(List(topic).asJava, listener) - - // rebalance to get the initial assignment - awaitRebalance(consumer, listener) - - val callsToAssignedAfterFirstRebalance = listener.callsToAssigned - - consumer.poll(Duration.ofMillis(2000)) - - // If the poll poll above times out, it would trigger a rebalance. - // Leave some time for the rebalance to happen and check for the rebalance event. - consumer.poll(Duration.ofMillis(500)) - consumer.poll(Duration.ofMillis(500)) - - assertEquals(callsToAssignedAfterFirstRebalance, listener.callsToAssigned) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testAutoCommitOnClose(quorum: String, groupProtocol: String): Unit = { @@ -363,155 +211,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { consumeAndVerifyRecords(consumer = consumer, numRecords = 1, startingOffset = 0, startingTimestamp = startingTimestamp) } - /** - * Verifies that pattern subscription performs as expected. - * The pattern matches the topics 'topic' and 'tblablac', but not 'tblablak' or 'tblab1'. - * It is expected that the consumer is subscribed to all partitions of 'topic' and - * 'tblablac' after the subscription when metadata is refreshed. - * When a new topic 'tsomec' is added afterwards, it is expected that upon the next - * metadata refresh the consumer becomes subscribed to this new topic and all partitions - * of that topic are assigned to it. - */ - // TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented. - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) - def testPatternSubscription(quorum: String, groupProtocol: String): Unit = { - val numRecords = 10000 - val producer = createProducer() - sendRecords(producer, numRecords, tp) - - val topic1 = "tblablac" // matches subscribed pattern - createTopic(topic1, 2, brokerCount) - sendRecords(producer, numRecords = 1000, new TopicPartition(topic1, 0)) - sendRecords(producer, numRecords = 1000, new TopicPartition(topic1, 1)) - - val topic2 = "tblablak" // does not match subscribed pattern - createTopic(topic2, 2, brokerCount) - sendRecords(producer,numRecords = 1000, new TopicPartition(topic2, 0)) - sendRecords(producer, numRecords = 1000, new TopicPartition(topic2, 1)) - - val topic3 = "tblab1" // does not match subscribed pattern - createTopic(topic3, 2, brokerCount) - sendRecords(producer, numRecords = 1000, new TopicPartition(topic3, 0)) - sendRecords(producer, numRecords = 1000, new TopicPartition(topic3, 1)) - - val consumer = createConsumer() - assertEquals(0, consumer.assignment().size) - - val pattern = Pattern.compile("t.*c") - consumer.subscribe(pattern, new TestConsumerReassignmentListener) - - var assignment = Set( - new TopicPartition(topic, 0), - new TopicPartition(topic, 1), - new TopicPartition(topic1, 0), - new TopicPartition(topic1, 1)) - awaitAssignment(consumer, assignment) - - val topic4 = "tsomec" // matches subscribed pattern - createTopic(topic4, 2, brokerCount) - sendRecords(producer, numRecords = 1000, new TopicPartition(topic4, 0)) - sendRecords(producer, numRecords = 1000, new TopicPartition(topic4, 1)) - - assignment ++= Set( - new TopicPartition(topic4, 0), - new TopicPartition(topic4, 1)) - awaitAssignment(consumer, assignment) - - consumer.unsubscribe() - assertEquals(0, consumer.assignment().size) - } - - /** - * Verifies that a second call to pattern subscription succeeds and performs as expected. - * The initial subscription is to a pattern that matches two topics 'topic' and 'foo'. - * The second subscription is to a pattern that matches 'foo' and a new topic 'bar'. - * It is expected that the consumer is subscribed to all partitions of 'topic' and 'foo' after - * the first subscription, and to all partitions of 'foo' and 'bar' after the second. - * The metadata refresh interval is intentionally increased to a large enough value to guarantee - * that it is the subscription call that triggers a metadata refresh, and not the timeout. - */ - // TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented. - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) - def testSubsequentPatternSubscription(quorum: String, groupProtocol: String): Unit = { - this.consumerConfig.setProperty(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "30000") - val consumer = createConsumer() - - val numRecords = 10000 - val producer = createProducer() - sendRecords(producer, numRecords = numRecords, tp) - - // the first topic ('topic') matches first subscription pattern only - - val fooTopic = "foo" // matches both subscription patterns - createTopic(fooTopic, 1, brokerCount) - sendRecords(producer, numRecords = 1000, new TopicPartition(fooTopic, 0)) - - assertEquals(0, consumer.assignment().size) - - val pattern1 = Pattern.compile(".*o.*") // only 'topic' and 'foo' match this - consumer.subscribe(pattern1, new TestConsumerReassignmentListener) - - var assignment = Set( - new TopicPartition(topic, 0), - new TopicPartition(topic, 1), - new TopicPartition(fooTopic, 0)) - awaitAssignment(consumer, assignment) - - val barTopic = "bar" // matches the next subscription pattern - createTopic(barTopic, 1, brokerCount) - sendRecords(producer, numRecords = 1000, new TopicPartition(barTopic, 0)) - - val pattern2 = Pattern.compile("...") // only 'foo' and 'bar' match this - consumer.subscribe(pattern2, new TestConsumerReassignmentListener) - assignment --= Set( - new TopicPartition(topic, 0), - new TopicPartition(topic, 1)) - assignment ++= Set( - new TopicPartition(barTopic, 0)) - awaitAssignment(consumer, assignment) - - consumer.unsubscribe() - assertEquals(0, consumer.assignment().size) - } - - /** - * Verifies that pattern unsubscription performs as expected. - * The pattern matches the topics 'topic' and 'tblablac'. - * It is expected that the consumer is subscribed to all partitions of 'topic' and - * 'tblablac' after the subscription when metadata is refreshed. - * When consumer unsubscribes from all its subscriptions, it is expected that its - * assignments are cleared right away. - */ - // TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented. - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) - def testPatternUnsubscription(quorum: String, groupProtocol: String): Unit = { - val numRecords = 10000 - val producer = createProducer() - sendRecords(producer, numRecords, tp) - - val topic1 = "tblablac" // matches the subscription pattern - createTopic(topic1, 2, brokerCount) - sendRecords(producer, numRecords = 1000, new TopicPartition(topic1, 0)) - sendRecords(producer, numRecords = 1000, new TopicPartition(topic1, 1)) - - val consumer = createConsumer() - assertEquals(0, consumer.assignment().size) - - consumer.subscribe(Pattern.compile("t.*c"), new TestConsumerReassignmentListener) - val assignment = Set( - new TopicPartition(topic, 0), - new TopicPartition(topic, 1), - new TopicPartition(topic1, 0), - new TopicPartition(topic1, 1)) - awaitAssignment(consumer, assignment) - - consumer.unsubscribe() - assertEquals(0, consumer.assignment().size) - } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testCommitMetadata(quorum: String, groupProtocol: String): Unit = { @@ -554,36 +253,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(new OffsetAndMetadata(count), consumer.committed(Set(tp).asJava).get(tp)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testExpandingTopicSubscriptions(quorum: String, groupProtocol: String): Unit = { - val otherTopic = "other" - val initialAssignment = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) - val consumer = createConsumer() - consumer.subscribe(List(topic).asJava) - awaitAssignment(consumer, initialAssignment) - - createTopic(otherTopic, 2, brokerCount) - val expandedAssignment = initialAssignment ++ Set(new TopicPartition(otherTopic, 0), new TopicPartition(otherTopic, 1)) - consumer.subscribe(List(topic, otherTopic).asJava) - awaitAssignment(consumer, expandedAssignment) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testShrinkingTopicSubscriptions(quorum: String, groupProtocol: String): Unit = { - val otherTopic = "other" - createTopic(otherTopic, 2, brokerCount) - val initialAssignment = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1), new TopicPartition(otherTopic, 0), new TopicPartition(otherTopic, 1)) - val consumer = createConsumer() - consumer.subscribe(List(topic, otherTopic).asJava) - awaitAssignment(consumer, initialAssignment) - - val shrunkenAssignment = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) - consumer.subscribe(List(topic).asJava) - awaitAssignment(consumer, shrunkenAssignment) - } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testPartitionsFor(quorum: String, groupProtocol: String): Unit = { @@ -721,583 +390,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { consumeAndVerifyRecords(consumer = consumer, numRecords = 5, startingOffset = 5, startingTimestamp = startingTimestamp) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testFetchInvalidOffset(quorum: String, groupProtocol: String): Unit = { - this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") - val consumer = createConsumer(configOverrides = this.consumerConfig) - - // produce one record - val totalRecords = 2 - val producer = createProducer() - sendRecords(producer, totalRecords, tp) - consumer.assign(List(tp).asJava) - - // poll should fail because there is no offset reset strategy set. - // we fail only when resetting positions after coordinator is known, so using a long timeout. - assertThrows(classOf[NoOffsetForPartitionException], () => consumer.poll(Duration.ofMillis(15000))) - - // seek to out of range position - val outOfRangePos = totalRecords + 1 - consumer.seek(tp, outOfRangePos) - val e = assertThrows(classOf[OffsetOutOfRangeException], () => consumer.poll(Duration.ofMillis(20000))) - val outOfRangePartitions = e.offsetOutOfRangePartitions() - assertNotNull(outOfRangePartitions) - assertEquals(1, outOfRangePartitions.size) - assertEquals(outOfRangePos.toLong, outOfRangePartitions.get(tp)) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testFetchOutOfRangeOffsetResetConfigEarliest(quorum: String, groupProtocol: String): Unit = { - this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") - // ensure no in-flight fetch request so that the offset can be reset immediately - this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG, "0") - val consumer = createConsumer(configOverrides = this.consumerConfig) - val totalRecords = 10L - - val producer = createProducer() - val startingTimestamp = 0 - sendRecords(producer, totalRecords.toInt, tp, startingTimestamp = startingTimestamp) - consumer.assign(List(tp).asJava) - consumeAndVerifyRecords(consumer = consumer, numRecords = totalRecords.toInt, startingOffset =0) - // seek to out of range position - val outOfRangePos = totalRecords + 1 - consumer.seek(tp, outOfRangePos) - // assert that poll resets to the beginning position - consumeAndVerifyRecords(consumer = consumer, numRecords = 1, startingOffset = 0) - } - - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testFetchOutOfRangeOffsetResetConfigLatest(quorum: String, groupProtocol: String): Unit = { - this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest") - // ensure no in-flight fetch request so that the offset can be reset immediately - this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG, "0") - val consumer = createConsumer(configOverrides = this.consumerConfig) - val totalRecords = 10L - - val producer = createProducer() - val startingTimestamp = 0 - sendRecords(producer, totalRecords.toInt, tp, startingTimestamp = startingTimestamp) - consumer.assign(List(tp).asJava) - consumer.seek(tp, 0) - // consume some, but not all of the records - consumeAndVerifyRecords(consumer = consumer, numRecords = totalRecords.toInt/2, startingOffset = 0) - // seek to out of range position - val outOfRangePos = totalRecords + 17 // arbitrary, much higher offset - consumer.seek(tp, outOfRangePos) - // assert that poll resets to the ending position - assertTrue(consumer.poll(Duration.ofMillis(50)).isEmpty) - sendRecords(producer, totalRecords.toInt, tp, startingTimestamp = totalRecords) - val nextRecord = consumer.poll(Duration.ofMillis(50)).iterator().next() - // ensure the seek went to the last known record at the time of the previous poll - assertEquals(totalRecords, nextRecord.offset()) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testFetchRecordLargerThanFetchMaxBytes(quorum: String, groupProtocol: String): Unit = { - val maxFetchBytes = 10 * 1024 - this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, maxFetchBytes.toString) - checkLargeRecord(maxFetchBytes + 1) - } - - private def checkLargeRecord(producerRecordSize: Int): Unit = { - val consumer = createConsumer() - - // produce a record that is larger than the configured fetch size - val record = new ProducerRecord(tp.topic(), tp.partition(), "key".getBytes, - new Array[Byte](producerRecordSize)) - val producer = createProducer() - producer.send(record) - - // consuming a record that is too large should succeed since KIP-74 - consumer.assign(List(tp).asJava) - val records = consumer.poll(Duration.ofMillis(20000)) - assertEquals(1, records.count) - val consumerRecord = records.iterator().next() - assertEquals(0L, consumerRecord.offset) - assertEquals(tp.topic(), consumerRecord.topic()) - assertEquals(tp.partition(), consumerRecord.partition()) - assertArrayEquals(record.key(), consumerRecord.key()) - assertArrayEquals(record.value(), consumerRecord.value()) - } - - /** We should only return a large record if it's the first record in the first non-empty partition of the fetch request */ - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testFetchHonoursFetchSizeIfLargeRecordNotFirst(quorum: String, groupProtocol: String): Unit = { - val maxFetchBytes = 10 * 1024 - this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, maxFetchBytes.toString) - checkFetchHonoursSizeIfLargeRecordNotFirst(maxFetchBytes) - } - - private def checkFetchHonoursSizeIfLargeRecordNotFirst(largeProducerRecordSize: Int): Unit = { - val consumer = createConsumer() - - val smallRecord = new ProducerRecord(tp.topic(), tp.partition(), "small".getBytes, - "value".getBytes) - val largeRecord = new ProducerRecord(tp.topic(), tp.partition(), "large".getBytes, - new Array[Byte](largeProducerRecordSize)) - - val producer = createProducer() - producer.send(smallRecord).get - producer.send(largeRecord).get - - // we should only get the small record in the first `poll` - consumer.assign(List(tp).asJava) - val records = consumer.poll(Duration.ofMillis(20000)) - assertEquals(1, records.count) - val consumerRecord = records.iterator().next() - assertEquals(0L, consumerRecord.offset) - assertEquals(tp.topic(), consumerRecord.topic()) - assertEquals(tp.partition(), consumerRecord.partition()) - assertArrayEquals(smallRecord.key(), consumerRecord.key()) - assertArrayEquals(smallRecord.value(), consumerRecord.value()) - } - - /** We should only return a large record if it's the first record in the first partition of the fetch request */ - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testFetchHonoursMaxPartitionFetchBytesIfLargeRecordNotFirst(quorum: String, groupProtocol: String): Unit = { - val maxPartitionFetchBytes = 10 * 1024 - this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, maxPartitionFetchBytes.toString) - checkFetchHonoursSizeIfLargeRecordNotFirst(maxPartitionFetchBytes) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testFetchRecordLargerThanMaxPartitionFetchBytes(quorum: String, groupProtocol: String): Unit = { - val maxPartitionFetchBytes = 10 * 1024 - this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, maxPartitionFetchBytes.toString) - checkLargeRecord(maxPartitionFetchBytes + 1) - } - - /** Test that we consume all partitions if fetch max bytes and max.partition.fetch.bytes are low */ - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testLowMaxFetchSizeForRequestAndPartition(quorum: String, groupProtocol: String): Unit = { - // one of the effects of this is that there will be some log reads where `0 > remaining limit bytes < message size` - // and we don't return the message because it's not the first message in the first non-empty partition of the fetch - // this behaves a little different than when remaining limit bytes is 0 and it's important to test it - this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, "500") - this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, "100") - - // Avoid a rebalance while the records are being sent (the default is 6 seconds) - this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 20000.toString) - val consumer = createConsumer() - - val topic1 = "topic1" - val topic2 = "topic2" - val topic3 = "topic3" - val partitionCount = 30 - val topics = Seq(topic1, topic2, topic3) - topics.foreach { topicName => - createTopic(topicName, partitionCount, brokerCount) - } - - val partitions = topics.flatMap { topic => - (0 until partitionCount).map(new TopicPartition(topic, _)) - } - - assertEquals(0, consumer.assignment().size) - - consumer.subscribe(List(topic1, topic2, topic3).asJava) - - awaitAssignment(consumer, partitions.toSet) - - val producer = createProducer() - - val producerRecords = partitions.flatMap(sendRecords(producer, numRecords = partitionCount, _)) - - val consumerRecords = consumeRecords(consumer, producerRecords.size) - - val expected = producerRecords.map { record => - (record.topic, record.partition, new String(record.key), new String(record.value), record.timestamp) - }.toSet - - val actual = consumerRecords.map { record => - (record.topic, record.partition, new String(record.key), new String(record.value), record.timestamp) - }.toSet - - assertEquals(expected, actual) - } - - - // Remote assignors only supported with consumer group protocol - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @CsvSource(Array( - "kraft+kip848, consumer" - )) - def testRemoteAssignorInvalid(quorum: String, groupProtocol: String): Unit = { - // 1 consumer using invalid remote assignor - this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "invalid-assignor-group") - this.consumerConfig.setProperty(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "invalid") - val consumer = createConsumer() - - // create two new topics, each having 2 partitions - val topic1 = "topic1" - val producer = createProducer() - val expectedAssignment = createTopicAndSendRecords(producer, topic1, 2, 100) - - assertEquals(0, consumer.assignment().size) - - // subscribe to two topics - consumer.subscribe(List(topic1).asJava) - - val e:UnsupportedAssignorException = assertThrows( - classOf[UnsupportedAssignorException], - () => awaitAssignment(consumer, expectedAssignment) - ) - - assertTrue(e.getMessage.startsWith("ServerAssignor invalid is not supported. " + - "Supported assignors: ")) - } - - // Remote assignors only supported with consumer group protocol - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @CsvSource(Array( - "kraft+kip848, consumer" - )) - def testRemoteAssignorRange(quorum: String, groupProtocol: String): Unit = { - // 1 consumer using range assignment - this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "range-group") - this.consumerConfig.setProperty(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "range") - this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, "30000") - val consumer = createConsumer() - - // create two new topics, each having 2 partitions - val topic1 = "topic1" - val topic2 = "topic2" - val producer = createProducer() - val expectedAssignment = createTopicAndSendRecords(producer, topic1, 2, 100) ++ - createTopicAndSendRecords(producer, topic2, 2, 100) - - assertEquals(0, consumer.assignment().size) - - // subscribe to two topics - consumer.subscribe(List(topic1, topic2).asJava) - awaitAssignment(consumer, expectedAssignment) - - // add one more topic with 2 partitions - val topic3 = "topic3" - val additionalAssignment = createTopicAndSendRecords(producer, topic3, 2, 100) - - val newExpectedAssignment = expectedAssignment ++ additionalAssignment - consumer.subscribe(List(topic1, topic2, topic3).asJava) - awaitAssignment(consumer, newExpectedAssignment) - - // remove the topic we just added - consumer.subscribe(List(topic1, topic2).asJava) - awaitAssignment(consumer, expectedAssignment) - - consumer.unsubscribe() - assertEquals(0, consumer.assignment().size) - } - - // Only the classic group protocol supports client-side assignors - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) - def testRoundRobinAssignment(quorum: String, groupProtocol: String): Unit = { - // 1 consumer using round-robin assignment - this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "roundrobin-group") - this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[RoundRobinAssignor].getName) - val consumer = createConsumer() - - // create two new topics, each having 2 partitions - val topic1 = "topic1" - val topic2 = "topic2" - val producer = createProducer() - val expectedAssignment = createTopicAndSendRecords(producer, topic1, 2, 100) ++ - createTopicAndSendRecords(producer, topic2, 2, 100) - - assertEquals(0, consumer.assignment().size) - - // subscribe to two topics - consumer.subscribe(List(topic1, topic2).asJava) - awaitAssignment(consumer, expectedAssignment) - - // add one more topic with 2 partitions - val topic3 = "topic3" - createTopicAndSendRecords(producer, topic3, 2, 100) - - val newExpectedAssignment = expectedAssignment ++ Set(new TopicPartition(topic3, 0), new TopicPartition(topic3, 1)) - consumer.subscribe(List(topic1, topic2, topic3).asJava) - awaitAssignment(consumer, newExpectedAssignment) - - // remove the topic we just added - consumer.subscribe(List(topic1, topic2).asJava) - awaitAssignment(consumer, expectedAssignment) - - consumer.unsubscribe() - assertEquals(0, consumer.assignment().size) - } - - // Only the classic group protocol supports client-side assignors - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) - def testMultiConsumerRoundRobinAssignor(quorum: String, groupProtocol: String): Unit = { - this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "roundrobin-group") - this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[RoundRobinAssignor].getName) - - // create two new topics, total number of partitions must be greater than number of consumers - val topic1 = "topic1" - val topic2 = "topic2" - val producer = createProducer() - val subscriptions = createTopicAndSendRecords(producer, topic1, 5, 100) ++ - createTopicAndSendRecords(producer, topic2, 8, 100) - - // create a group of consumers, subscribe the consumers to all the topics and start polling - // for the topic partition assignment - val (consumerGroup, consumerPollers) = createConsumerGroupAndWaitForAssignment(10, List(topic1, topic2), subscriptions) - try { - validateGroupAssignment(consumerPollers, subscriptions) - - // add one more consumer and validate re-assignment - addConsumersToGroupAndWaitForGroupAssignment(1, consumerGroup, consumerPollers, - List(topic1, topic2), subscriptions, "roundrobin-group") - } finally { - consumerPollers.foreach(_.shutdown()) - } - } - - /** - * This test runs the following scenario to verify sticky assignor behavior. - * Topics: single-topic, with random number of partitions, where #par is 10, 20, 30, 40, 50, 60, 70, 80, 90, or 100 - * Consumers: 9 consumers subscribed to the single topic - * Expected initial assignment: partitions are assigned to consumers in a round robin fashion. - * - (#par mod 9) consumers will get (#par / 9 + 1) partitions, and the rest get (#par / 9) partitions - * Then consumer #10 is added to the list (subscribing to the same single topic) - * Expected new assignment: - * - (#par / 10) partition per consumer, where one partition from each of the early (#par mod 9) consumers - * will move to consumer #10, leading to a total of (#par mod 9) partition movement - */ - // Only the classic group protocol supports client-side assignors - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) - def testMultiConsumerStickyAssignor(quorum: String, groupProtocol: String): Unit = { - - def reverse(m: Map[Long, Set[TopicPartition]]) = - m.values.toSet.flatten.map(v => (v, m.keys.filter(m(_).contains(v)).head)).toMap - - this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "sticky-group") - this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[StickyAssignor].getName) - - // create one new topic - val topic = "single-topic" - val rand = 1 + scala.util.Random.nextInt(10) - val producer = createProducer() - val partitions = createTopicAndSendRecords(producer, topic, rand * 10, 100) - - // create a group of consumers, subscribe the consumers to the single topic and start polling - // for the topic partition assignment - val (consumerGroup, consumerPollers) = createConsumerGroupAndWaitForAssignment(9, List(topic), partitions) - validateGroupAssignment(consumerPollers, partitions) - val prePartition2PollerId = reverse(consumerPollers.map(poller => (poller.getId, poller.consumerAssignment())).toMap) - - // add one more consumer and validate re-assignment - addConsumersToGroupAndWaitForGroupAssignment(1, consumerGroup, consumerPollers, List(topic), partitions, "sticky-group") - - val postPartition2PollerId = reverse(consumerPollers.map(poller => (poller.getId, poller.consumerAssignment())).toMap) - val keys = prePartition2PollerId.keySet.union(postPartition2PollerId.keySet) - var changes = 0 - keys.foreach { key => - val preVal = prePartition2PollerId.get(key) - val postVal = postPartition2PollerId.get(key) - if (preVal.nonEmpty && postVal.nonEmpty) { - if (preVal.get != postVal.get) - changes += 1 - } else - changes += 1 - } - - consumerPollers.foreach(_.shutdown()) - - assertEquals(rand, changes, "Expected only two topic partitions that have switched to other consumers.") - } - - /** - * This test re-uses BaseConsumerTest's consumers. - * As a result, it is testing the default assignment strategy set by BaseConsumerTest - */ - // Only the classic group protocol supports client-side assignors - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) - def testMultiConsumerDefaultAssignor(quorum: String, groupProtocol: String): Unit = { - // use consumers and topics defined in this class + one more topic - val producer = createProducer() - sendRecords(producer, numRecords = 100, tp) - sendRecords(producer, numRecords = 100, tp2) - val topic1 = "topic1" - val subscriptions = Set(tp, tp2) ++ createTopicAndSendRecords(producer, topic1, 5, 100) - - // subscribe all consumers to all topics and validate the assignment - - val consumersInGroup = Buffer[Consumer[Array[Byte], Array[Byte]]]() - consumersInGroup += createConsumer() - consumersInGroup += createConsumer() - - val consumerPollers = subscribeConsumers(consumersInGroup, List(topic, topic1)) - try { - validateGroupAssignment(consumerPollers, subscriptions) - - // add 2 more consumers and validate re-assignment - addConsumersToGroupAndWaitForGroupAssignment(2, consumersInGroup, consumerPollers, List(topic, topic1), subscriptions) - - // add one more topic and validate partition re-assignment - val topic2 = "topic2" - val expandedSubscriptions = subscriptions ++ createTopicAndSendRecords(producer, topic2, 3, 100) - changeConsumerGroupSubscriptionAndValidateAssignment(consumerPollers, List(topic, topic1, topic2), expandedSubscriptions) - - // remove the topic we just added and validate re-assignment - changeConsumerGroupSubscriptionAndValidateAssignment(consumerPollers, List(topic, topic1), subscriptions) - - } finally { - consumerPollers.foreach(_.shutdown()) - } - } - - // Only the classic group protocol supports client-side assignors - @ParameterizedTest - @CsvSource(Array( - "org.apache.kafka.clients.consumer.CooperativeStickyAssignor, zk", - "org.apache.kafka.clients.consumer.RangeAssignor, zk", - "org.apache.kafka.clients.consumer.CooperativeStickyAssignor, kraft", - "org.apache.kafka.clients.consumer.RangeAssignor, kraft" - )) - def testRebalanceAndRejoin(assignmentStrategy: String, quorum: String): Unit = { - // create 2 consumers - this.consumerConfig.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, "classic") - this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "rebalance-and-rejoin-group") - this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, assignmentStrategy) - this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true") - val consumer1 = createConsumer() - val consumer2 = createConsumer() - - // create a new topic, have 2 partitions - val topic = "topic1" - val producer = createProducer() - val expectedAssignment = createTopicAndSendRecords(producer, topic, 2, 100) - - assertEquals(0, consumer1.assignment().size) - assertEquals(0, consumer2.assignment().size) - - val lock = new ReentrantLock() - var generationId1 = -1 - var memberId1 = "" - val customRebalanceListener = new ConsumerRebalanceListener { - override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = { - } - override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]): Unit = { - if (!lock.tryLock(3000, TimeUnit.MILLISECONDS)) { - fail(s"Time out while awaiting for lock.") - } - try { - generationId1 = consumer1.groupMetadata().generationId() - memberId1 = consumer1.groupMetadata().memberId() - } finally { - lock.unlock() - } - } - } - val consumerPoller1 = new ConsumerAssignmentPoller(consumer1, List(topic), Set.empty, customRebalanceListener) - consumerPoller1.start() - TestUtils.waitUntilTrue(() => consumerPoller1.consumerAssignment() == expectedAssignment, - s"Timed out while awaiting expected assignment change to $expectedAssignment.") - - // Since the consumer1 already completed the rebalance, - // the `onPartitionsAssigned` rebalance listener will be invoked to set the generationId and memberId - var stableGeneration = -1 - var stableMemberId1 = "" - if (!lock.tryLock(3000, TimeUnit.MILLISECONDS)) { - fail(s"Time out while awaiting for lock.") - } - try { - stableGeneration = generationId1 - stableMemberId1 = memberId1 - } finally { - lock.unlock() - } - - val consumerPoller2 = subscribeConsumerAndStartPolling(consumer2, List(topic)) - TestUtils.waitUntilTrue(() => consumerPoller1.consumerAssignment().size == 1, - s"Timed out while awaiting expected assignment size change to 1.") - TestUtils.waitUntilTrue(() => consumerPoller2.consumerAssignment().size == 1, - s"Timed out while awaiting expected assignment size change to 1.") - - if (!lock.tryLock(3000, TimeUnit.MILLISECONDS)) { - fail(s"Time out while awaiting for lock.") - } - try { - if (assignmentStrategy.equals(classOf[CooperativeStickyAssignor].getName)) { - // cooperative rebalance should rebalance twice before finally stable - assertEquals(stableGeneration + 2, generationId1) - } else { - // eager rebalance should rebalance once before finally stable - assertEquals(stableGeneration + 1, generationId1) - } - assertEquals(stableMemberId1, memberId1) - } finally { - lock.unlock() - } - - consumerPoller1.shutdown() - consumerPoller2.shutdown() - } - - /** - * This test re-uses BaseConsumerTest's consumers. - * As a result, it is testing the default assignment strategy set by BaseConsumerTest - * It tests the assignment results is expected using default assignor (i.e. Range assignor) - */ - // Only the classic group protocol supports client-side assignors - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) - def testMultiConsumerDefaultAssignorAndVerifyAssignment(quorum: String, groupProtocol: String): Unit = { - // create two new topics, each having 3 partitions - val topic1 = "topic1" - val topic2 = "topic2" - - createTopic(topic1, 3) - createTopic(topic2, 3) - - val consumersInGroup = Buffer[Consumer[Array[Byte], Array[Byte]]]() - consumersInGroup += createConsumer() - consumersInGroup += createConsumer() - - val tp1_0 = new TopicPartition(topic1, 0) - val tp1_1 = new TopicPartition(topic1, 1) - val tp1_2 = new TopicPartition(topic1, 2) - val tp2_0 = new TopicPartition(topic2, 0) - val tp2_1 = new TopicPartition(topic2, 1) - val tp2_2 = new TopicPartition(topic2, 2) - - val subscriptions = Set(tp1_0, tp1_1, tp1_2, tp2_0, tp2_1, tp2_2) - val consumerPollers = subscribeConsumers(consumersInGroup, List(topic1, topic2)) - - val expectedAssignment = Buffer(Set(tp1_0, tp1_1, tp2_0, tp2_1), Set(tp1_2, tp2_2)) - - try { - validateGroupAssignment(consumerPollers, subscriptions, expectedAssignment = expectedAssignment) - } finally { - consumerPollers.foreach(_.shutdown()) - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testMultiConsumerSessionTimeoutOnStopPolling(quorum: String, groupProtocol: String): Unit = { - runMultiConsumerSessionTimeoutTest(false) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testMultiConsumerSessionTimeoutOnClose(quorum: String, groupProtocol: String): Unit = { - runMultiConsumerSessionTimeoutTest(true) - } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testInterceptors(quorum: String, groupProtocol: String): Unit = { @@ -1512,23 +604,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(2, topics.get(topic3).size) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testUnsubscribeTopic(quorum: String, groupProtocol: String): Unit = { - this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100") // timeout quickly to avoid slow test - this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30") - val consumer = createConsumer() - - val listener = new TestConsumerReassignmentListener() - consumer.subscribe(List(topic).asJava, listener) - - // the initial subscription should cause a callback execution - awaitRebalance(consumer, listener) - - consumer.subscribe(List[String]().asJava) - assertEquals(0, consumer.assignment.size()) - } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testPauseStateNotPreservedByRebalance(quorum: String, groupProtocol: String): Unit = { @@ -1786,53 +861,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertNotNull(fetchLag) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testPerPartitionLeadWithMaxPollRecords(quorum: String, groupProtocol: String): Unit = { - val numMessages = 1000 - val maxPollRecords = 10 - val producer = createProducer() - sendRecords(producer, numMessages, tp) - - consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testPerPartitionLeadWithMaxPollRecords") - consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLeadWithMaxPollRecords") - consumerConfig.setProperty(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString) - val consumer = createConsumer() - consumer.assign(List(tp).asJava) - awaitNonEmptyRecords(consumer, tp) - - val tags = new util.HashMap[String, String]() - tags.put("client-id", "testPerPartitionLeadWithMaxPollRecords") - tags.put("topic", tp.topic()) - tags.put("partition", String.valueOf(tp.partition())) - val lead = consumer.metrics.get(new MetricName("records-lead", "consumer-fetch-manager-metrics", "", tags)) - assertEquals(maxPollRecords, lead.metricValue().asInstanceOf[Double], s"The lead should be $maxPollRecords") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testPerPartitionLagWithMaxPollRecords(quorum: String, groupProtocol: String): Unit = { - val numMessages = 1000 - val maxPollRecords = 10 - val producer = createProducer() - sendRecords(producer, numMessages, tp) - - consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testPerPartitionLagWithMaxPollRecords") - consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLagWithMaxPollRecords") - consumerConfig.setProperty(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString) - val consumer = createConsumer() - consumer.assign(List(tp).asJava) - val records = awaitNonEmptyRecords(consumer, tp) - - 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(numMessages - records.count, lag.metricValue.asInstanceOf[Double], epsilon, s"The lag should be ${numMessages - records.count}") - } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testQuotaMetricsNotCreatedIfNoQuotasConfigured(quorum: String, groupProtocol: String): Unit = { @@ -1865,100 +893,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { brokers.foreach(assertNoMetric(_, "throttle-time", QuotaType.Request, consumerClientId)) } - def runMultiConsumerSessionTimeoutTest(closeConsumer: Boolean): Unit = { - // use consumers defined in this class plus one additional consumer - // Use topic defined in this class + one additional topic - val producer = createProducer() - sendRecords(producer, numRecords = 100, tp) - sendRecords(producer, numRecords = 100, tp2) - val topic1 = "topic1" - val subscriptions = Set(tp, tp2) ++ createTopicAndSendRecords(producer, topic1, 6, 100) - - // first subscribe consumers that are defined in this class - val consumerPollers = Buffer[ConsumerAssignmentPoller]() - consumerPollers += subscribeConsumerAndStartPolling(createConsumer(), List(topic, topic1)) - consumerPollers += subscribeConsumerAndStartPolling(createConsumer(), List(topic, topic1)) - - // create one more consumer and add it to the group; we will timeout this consumer - val timeoutConsumer = createConsumer() - val timeoutPoller = subscribeConsumerAndStartPolling(timeoutConsumer, List(topic, topic1)) - consumerPollers += timeoutPoller - - // validate the initial assignment - validateGroupAssignment(consumerPollers, subscriptions) - - // stop polling and close one of the consumers, should trigger partition re-assignment among alive consumers - timeoutPoller.shutdown() - consumerPollers -= timeoutPoller - if (closeConsumer) - timeoutConsumer.close() - - validateGroupAssignment(consumerPollers, subscriptions, - Some(s"Did not get valid assignment for partitions ${subscriptions.asJava} after one consumer left"), 3 * groupMaxSessionTimeoutMs) - - // done with pollers and consumers - for (poller <- consumerPollers) - poller.shutdown() - } - - /** - * Creates consumer pollers corresponding to a given consumer group, one per consumer; subscribes consumers to - * 'topicsToSubscribe' topics, waits until consumers get topics assignment. - * - * When the function returns, consumer pollers will continue to poll until shutdown is called on every poller. - * - * @param consumerGroup consumer group - * @param topicsToSubscribe topics to which consumers will subscribe to - * @return collection of consumer pollers - */ - def subscribeConsumers(consumerGroup: mutable.Buffer[Consumer[Array[Byte], Array[Byte]]], - topicsToSubscribe: List[String]): mutable.Buffer[ConsumerAssignmentPoller] = { - val consumerPollers = mutable.Buffer[ConsumerAssignmentPoller]() - for (consumer <- consumerGroup) - consumerPollers += subscribeConsumerAndStartPolling(consumer, topicsToSubscribe) - consumerPollers - } - - /** - * Creates 'consumerCount' consumers and consumer pollers, one per consumer; subscribes consumers to - * 'topicsToSubscribe' topics, waits until consumers get topics assignment. - * - * When the function returns, consumer pollers will continue to poll until shutdown is called on every poller. - * - * @param consumerCount number of consumers to create - * @param topicsToSubscribe topics to which consumers will subscribe to - * @param subscriptions set of all topic partitions - * @return collection of created consumers and collection of corresponding consumer pollers - */ - def createConsumerGroupAndWaitForAssignment(consumerCount: Int, - topicsToSubscribe: List[String], - subscriptions: Set[TopicPartition]): (Buffer[Consumer[Array[Byte], Array[Byte]]], Buffer[ConsumerAssignmentPoller]) = { - assertTrue(consumerCount <= subscriptions.size) - val consumerGroup = Buffer[Consumer[Array[Byte], Array[Byte]]]() - for (_ <- 0 until consumerCount) - consumerGroup += createConsumer() - - // create consumer pollers, wait for assignment and validate it - val consumerPollers = subscribeConsumers(consumerGroup, topicsToSubscribe) - (consumerGroup, consumerPollers) - } - - def changeConsumerGroupSubscriptionAndValidateAssignment(consumerPollers: Buffer[ConsumerAssignmentPoller], - topicsToSubscribe: List[String], - subscriptions: Set[TopicPartition]): Unit = { - for (poller <- consumerPollers) - poller.subscribe(topicsToSubscribe) - - // since subscribe call to poller does not actually call consumer subscribe right away, wait - // until subscribe is called on all consumers - TestUtils.waitUntilTrue(() => { - consumerPollers.forall { poller => poller.isSubscribeRequestProcessed } - }, s"Failed to call subscribe on all consumers in the group for subscription $subscriptions", 1000L) - - validateGroupAssignment(consumerPollers, subscriptions, - Some(s"Did not get valid assignment for partitions ${subscriptions.asJava} after we changed subscription")) - } - def changeConsumerSubscriptionAndValidateAssignment[K, V](consumer: Consumer[K, V], topicsToSubscribe: List[String], expectedAssignment: Set[TopicPartition], @@ -1967,21 +901,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { awaitAssignment(consumer, expectedAssignment) } - private def awaitNonEmptyRecords[K, V](consumer: Consumer[K, V], partition: TopicPartition): ConsumerRecords[K, V] = { - TestUtils.pollRecordsUntilTrue(consumer, (polledRecords: ConsumerRecords[K, V]) => { - if (polledRecords.records(partition).asScala.nonEmpty) - return polledRecords - false - }, s"Consumer did not consume any messages for partition $partition before timeout.") - throw new IllegalStateException("Should have timed out before reaching here") - } - - private def awaitAssignment(consumer: Consumer[_, _], expectedAssignment: Set[TopicPartition]): Unit = { - TestUtils.pollUntilTrue(consumer, () => consumer.assignment() == expectedAssignment.asJava, - s"Timed out while awaiting expected assignment $expectedAssignment. " + - s"The current assignment is ${consumer.assignment()}") - } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testConsumingWithNullGroupId(quorum: String, groupProtocol: String): Unit = { @@ -2149,211 +1068,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { consumer2.close() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAssignAndCommitAsyncNotCommitted(quorum:String, groupProtocol: String): Unit = { - val props = new Properties() - val consumer = createConsumer(configOverrides = props) - val producer = createProducer() - val numRecords = 10000 - val startingTimestamp = System.currentTimeMillis() - val cb = new CountConsumerCommitCallback - sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) - consumer.assign(List(tp).asJava) - consumer.commitAsync(cb) - TestUtils.pollUntilTrue(consumer, () => cb.successCount >= 1 || cb.lastError.isDefined, - "Failed to observe commit callback before timeout", waitTimeMs = 10000) - val committedOffset = consumer.committed(Set(tp).asJava) - assertNotNull(committedOffset) - // No valid fetch position due to the absence of consumer.poll; and therefore no offset was committed to - // tp. The committed offset should be null. This is intentional. - assertNull(committedOffset.get(tp)) - assertTrue(consumer.assignment.contains(tp)) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAssignAndCommitSyncNotCommitted(quorum:String, groupProtocol: String): Unit = { - val props = new Properties() - val consumer = createConsumer(configOverrides = props) - val producer = createProducer() - val numRecords = 10000 - val startingTimestamp = System.currentTimeMillis() - sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) - consumer.assign(List(tp).asJava) - consumer.commitSync() - val committedOffset = consumer.committed(Set(tp).asJava) - assertNotNull(committedOffset) - // No valid fetch position due to the absence of consumer.poll; and therefore no offset was committed to - // tp. The committed offset should be null. This is intentional. - assertNull(committedOffset.get(tp)) - assertTrue(consumer.assignment.contains(tp)) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAssignAndCommitSyncAllConsumed(quorum:String, groupProtocol: String): Unit = { - val numRecords = 10000 - - val producer = createProducer() - val startingTimestamp = System.currentTimeMillis() - sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) - - val props = new Properties() - val consumer = createConsumer(configOverrides = props) - consumer.assign(List(tp).asJava) - consumer.seek(tp, 0) - consumeAndVerifyRecords(consumer = consumer, numRecords, startingOffset = 0, startingTimestamp = startingTimestamp) - - consumer.commitSync() - val committedOffset = consumer.committed(Set(tp).asJava) - assertNotNull(committedOffset) - assertNotNull(committedOffset.get(tp)) - assertEquals(numRecords, committedOffset.get(tp).offset()) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAssignAndConsume(quorum:String, groupProtocol: String): Unit = { - val numRecords = 10 - - val producer = createProducer() - val startingTimestamp = System.currentTimeMillis() - sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) - - val props = new Properties() - val consumer = createConsumer(configOverrides = props, - configsToRemove = List(ConsumerConfig.GROUP_ID_CONFIG)) - consumer.assign(List(tp).asJava) - consumeAndVerifyRecords(consumer = consumer, numRecords, startingOffset = 0, startingTimestamp = startingTimestamp) - - assertEquals(numRecords, consumer.position(tp)) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAssignAndConsumeSkippingPosition(quorum:String, groupProtocol: String): Unit = { - val numRecords = 10 - - val producer = createProducer() - val startingTimestamp = System.currentTimeMillis() - sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) - - val props = new Properties() - val consumer = createConsumer(configOverrides = props, - configsToRemove = List(ConsumerConfig.GROUP_ID_CONFIG)) - consumer.assign(List(tp).asJava) - val offset = 1 - consumer.seek(tp, offset) - consumeAndVerifyRecords(consumer = consumer, numRecords - offset, startingOffset = offset, - startingKeyAndValueIndex = offset, startingTimestamp = startingTimestamp + offset) - - assertEquals(numRecords, consumer.position(tp)) - } - - // partitionsFor not implemented in consumer group protocol and this test requires ZK also - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @CsvSource(Array( - "zk, classic" - )) - def testAssignAndConsumeWithLeaderChangeValidatingPositions(quorum:String, groupProtocol: String): Unit = { - val numRecords = 10 - val producer = createProducer() - val startingTimestamp = System.currentTimeMillis() - sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) - val props = new Properties() - val consumer = createConsumer(configOverrides = props, - configsToRemove = List(ConsumerConfig.GROUP_ID_CONFIG)) - consumer.assign(List(tp).asJava) - consumeAndVerifyRecords(consumer = consumer, numRecords, startingOffset = 0, startingTimestamp = startingTimestamp) - - // Force leader epoch change to trigger position validation - var parts: mutable.Buffer[PartitionInfo] = null - while (parts == null) - parts = consumer.partitionsFor(tp.topic()).asScala - val leader = parts.head.leader().id() - this.servers(leader).shutdown() - this.servers(leader).startup() - - // Consume after leader change - sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) - consumeAndVerifyRecords(consumer = consumer, numRecords, startingOffset = 10, - startingTimestamp = startingTimestamp) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAssignAndFetchCommittedOffsets(quorum:String, groupProtocol: String): Unit = { - val numRecords = 100 - val startingTimestamp = System.currentTimeMillis() - val producer = createProducer() - sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) - val props = new Properties() - val consumer = createConsumer(configOverrides = props) - consumer.assign(List(tp).asJava) - // First consumer consumes and commits offsets - consumer.seek(tp, 0) - consumeAndVerifyRecords(consumer = consumer, numRecords, startingOffset = 0, - startingTimestamp = startingTimestamp) - consumer.commitSync() - assertEquals(numRecords, consumer.committed(Set(tp).asJava).get(tp).offset) - // We should see the committed offsets from another consumer - val anotherConsumer = createConsumer(configOverrides = props) - anotherConsumer.assign(List(tp).asJava) - assertEquals(numRecords, anotherConsumer.committed(Set(tp).asJava).get(tp).offset) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAssignAndConsumeFromCommittedOffsets(quorum:String, groupProtocol: String): Unit = { - val producer = createProducer() - val numRecords = 100 - val startingTimestamp = System.currentTimeMillis() - sendRecords(producer, numRecords = numRecords, tp, startingTimestamp = startingTimestamp) - - // Commit offset with first consumer - val props = new Properties() - props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "group1") - val consumer = createConsumer(configOverrides = props) - consumer.assign(List(tp).asJava) - val offset = 10 - consumer.commitSync(Map[TopicPartition, OffsetAndMetadata]((tp, new OffsetAndMetadata(offset))) - .asJava) - assertEquals(offset, consumer.committed(Set(tp).asJava).get(tp).offset) - consumer.close() - - // Consume from committed offsets with another consumer in same group - val anotherConsumer = createConsumer(configOverrides = props) - assertEquals(offset, anotherConsumer.committed(Set(tp).asJava).get(tp).offset) - anotherConsumer.assign(List(tp).asJava) - consumeAndVerifyRecords(consumer = anotherConsumer, numRecords - offset, - startingOffset = offset, startingKeyAndValueIndex = offset, - startingTimestamp = startingTimestamp + offset) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAssignAndRetrievingCommittedOffsetsMultipleTimes(quorum:String, groupProtocol: String): Unit = { - val numRecords = 100 - val startingTimestamp = System.currentTimeMillis() - val producer = createProducer() - sendRecords(producer, numRecords, tp, startingTimestamp = startingTimestamp) - - val props = new Properties() - val consumer = createConsumer(configOverrides = props) - consumer.assign(List(tp).asJava) - - // Consume and commit offsets - consumer.seek(tp, 0) - consumeAndVerifyRecords(consumer = consumer, numRecords, startingOffset = 0, - startingTimestamp = startingTimestamp) - consumer.commitSync() - - // Check committed offsets twice with same consumer - assertEquals(numRecords, consumer.committed(Set(tp).asJava).get(tp).offset) - assertEquals(numRecords, consumer.committed(Set(tp).asJava).get(tp).offset) - } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testSubscribeAndCommitSync(quorum: String, groupProtocol: String): Unit = { diff --git a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala index eba51c968f..11afbd89d2 100644 --- a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala @@ -23,7 +23,6 @@ import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.SaslAuthenticationException import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import org.junit.jupiter.api.Assertions._ -import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService} import kafka.server.KafkaConfig import kafka.utils.{JaasTestUtils, TestUtils} import kafka.zk.ConfigEntityChangeNotificationZNode @@ -31,7 +30,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource -class SaslClientsWithInvalidCredentialsTest extends IntegrationTestHarness with SaslSetup { +class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { private val kafkaClientSaslMechanism = "SCRAM-SHA-256" private val kafkaServerSaslMechanisms = List(kafkaClientSaslMechanism) override protected val securityProtocol = SecurityProtocol.SASL_PLAINTEXT @@ -166,45 +165,6 @@ class SaslClientsWithInvalidCredentialsTest extends IntegrationTestHarness with } } - @Test - def testConsumerGroupServiceWithAuthenticationFailure(): Unit = { - val consumerGroupService: ConsumerGroupService = prepareConsumerGroupService - - val consumer = createConsumer() - try { - consumer.subscribe(List(topic).asJava) - - verifyAuthenticationException(consumerGroupService.listGroups()) - } finally consumerGroupService.close() - } - - @Test - def testConsumerGroupServiceWithAuthenticationSuccess(): Unit = { - createClientCredential() - val consumerGroupService: ConsumerGroupService = prepareConsumerGroupService - - val consumer = createConsumer() - try { - consumer.subscribe(List(topic).asJava) - - verifyWithRetry(consumer.poll(Duration.ofMillis(1000))) - assertEquals(1, consumerGroupService.listConsumerGroups().size) - } - finally consumerGroupService.close() - } - - private def prepareConsumerGroupService = { - val propsFile = TestUtils.tempPropertiesFile(Map("security.protocol" -> "SASL_PLAINTEXT", "sasl.mechanism" -> kafkaClientSaslMechanism)) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), - "--describe", - "--group", "test.group", - "--command-config", propsFile.getAbsolutePath) - val opts = new ConsumerGroupCommandOptions(cgcArgs) - val consumerGroupService = new ConsumerGroupService(opts) - consumerGroupService - } - private def createClientCredential(): Unit = { createScramCredentialsViaPrivilegedAdminClient(JaasTestUtils.KafkaScramUser2, JaasTestUtils.KafkaScramPassword2) } diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 4be668aa32..c984eae027 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -254,7 +254,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup expectedProps.setProperty(KafkaConfig.LogRetentionTimeMillisProp, "1680000000") expectedProps.setProperty(KafkaConfig.LogRetentionTimeHoursProp, "168") expectedProps.setProperty(KafkaConfig.LogRollTimeHoursProp, "168") - expectedProps.setProperty(KafkaConfig.LogCleanerThreadsProp, "1") + expectedProps.setProperty(CleanerConfig.LOG_CLEANER_THREADS_PROP, "1") val logRetentionMs = configEntry(configDesc, KafkaConfig.LogRetentionTimeMillisProp) verifyConfig(KafkaConfig.LogRetentionTimeMillisProp, logRetentionMs, isSensitive = false, isReadOnly = false, expectedProps) @@ -264,8 +264,8 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup val logRollHours = configEntry(configDesc, KafkaConfig.LogRollTimeHoursProp) verifyConfig(KafkaConfig.LogRollTimeHoursProp, logRollHours, isSensitive = false, isReadOnly = true, expectedProps) - val logCleanerThreads = configEntry(configDesc, KafkaConfig.LogCleanerThreadsProp) - verifyConfig(KafkaConfig.LogCleanerThreadsProp, logCleanerThreads, + val logCleanerThreads = configEntry(configDesc, CleanerConfig.LOG_CLEANER_THREADS_PROP) + verifyConfig(CleanerConfig.LOG_CLEANER_THREADS_PROP, logCleanerThreads, isSensitive = false, isReadOnly = false, expectedProps) def synonymsList(configEntry: ConfigEntry): List[(String, ConfigSource)] = @@ -278,7 +278,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup (KafkaConfig.LogRetentionTimeHoursProp, ConfigSource.DEFAULT_CONFIG)), synonymsList(logRetentionHours)) assertEquals(List((KafkaConfig.LogRollTimeHoursProp, ConfigSource.DEFAULT_CONFIG)), synonymsList(logRollHours)) - assertEquals(List((KafkaConfig.LogCleanerThreadsProp, ConfigSource.DEFAULT_CONFIG)), synonymsList(logCleanerThreads)) + assertEquals(List((CleanerConfig.LOG_CLEANER_THREADS_PROP, ConfigSource.DEFAULT_CONFIG)), synonymsList(logCleanerThreads)) } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @@ -536,19 +536,19 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup verifyThreads("kafka-log-cleaner-thread-", countPerBroker = 1) val props = new Properties - props.put(KafkaConfig.LogCleanerThreadsProp, "2") - props.put(KafkaConfig.LogCleanerDedupeBufferSizeProp, "20000000") - props.put(KafkaConfig.LogCleanerDedupeBufferLoadFactorProp, "0.8") - props.put(KafkaConfig.LogCleanerIoBufferSizeProp, "300000") + props.put(CleanerConfig.LOG_CLEANER_THREADS_PROP, "2") + props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "20000000") + props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, "0.8") + props.put(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, "300000") props.put(KafkaConfig.MessageMaxBytesProp, "40000") - props.put(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp, "50000000") - props.put(KafkaConfig.LogCleanerBackoffMsProp, "6000") + props.put(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, "50000000") + props.put(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, "6000") // Verify cleaner config was updated. Wait for one of the configs to be updated and verify // that all other others were updated at the same time since they are reconfigured together var newCleanerConfig: CleanerConfig = null TestUtils.waitUntilTrue(() => { - reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogCleanerThreadsProp, "2")) + reconfigureServers(props, perBrokerConfig = false, (CleanerConfig.LOG_CLEANER_THREADS_PROP, "2")) newCleanerConfig = servers.head.logManager.cleaner.currentConfig newCleanerConfig.numThreads == 2 }, "Log cleaner not reconfigured", 60000) @@ -566,8 +566,8 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup def cleanerThreads = Thread.getAllStackTraces.keySet.asScala.filter(_.getName.startsWith("kafka-log-cleaner-thread-")) cleanerThreads.take(2).foreach(_.interrupt()) TestUtils.waitUntilTrue(() => cleanerThreads.size == (2 * numServers) - 2, "Threads did not exit") - props.put(KafkaConfig.LogCleanerBackoffMsProp, "8000") - reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogCleanerBackoffMsProp, "8000")) + props.put(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, "8000") + reconfigureServers(props, perBrokerConfig = false, (CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, "8000")) verifyThreads("kafka-log-cleaner-thread-", countPerBroker = 2) // Verify that produce/consume worked throughout this test without any retries in producer @@ -635,10 +635,10 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props.put(KafkaConfig.LogRetentionTimeMillisProp, TimeUnit.DAYS.toMillis(1).toString) props.put(KafkaConfig.MessageMaxBytesProp, "100000") props.put(KafkaConfig.LogIndexIntervalBytesProp, "10000") - props.put(KafkaConfig.LogCleanerDeleteRetentionMsProp, TimeUnit.DAYS.toMillis(1).toString) - props.put(KafkaConfig.LogCleanerMinCompactionLagMsProp, "60000") + props.put(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, TimeUnit.DAYS.toMillis(1).toString) + props.put(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP, "60000") props.put(KafkaConfig.LogDeleteDelayMsProp, "60000") - props.put(KafkaConfig.LogCleanerMinCleanRatioProp, "0.3") + props.put(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP, "0.3") props.put(KafkaConfig.LogCleanupPolicyProp, "delete") props.put(KafkaConfig.UncleanLeaderElectionEnableProp, "false") props.put(KafkaConfig.MinInSyncReplicasProp, "2") diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 130d0e5642..3be1b400ab 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.clients.admin._ import org.apache.kafka.common.acl.{AclBinding, AclBindingFilter} import org.apache.kafka.common.config.{ConfigException, ConfigResource} import org.apache.kafka.common.config.ConfigResource.Type -import org.apache.kafka.common.errors.{PolicyViolationException, UnsupportedVersionException} +import org.apache.kafka.common.errors.{InvalidPartitionsException,PolicyViolationException, UnsupportedVersionException} import org.apache.kafka.common.message.DescribeClusterRequestData import org.apache.kafka.common.metadata.{ConfigRecord, FeatureLevelRecord} import org.apache.kafka.common.metrics.Metrics @@ -792,6 +792,39 @@ class KRaftClusterTest { } } + @ParameterizedTest + @ValueSource(strings = Array("3.7-IV0", "3.7-IV2")) + def testCreatePartitions(metadataVersionString: String): Unit = { + val cluster = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(4). + setBootstrapMetadataVersion(MetadataVersion.fromVersionString(metadataVersionString)). + setNumControllerNodes(3).build()). + build() + try { + cluster.format() + cluster.startup() + cluster.waitForReadyBrokers() + val admin = Admin.create(cluster.clientProperties()) + try { + val createResults = admin.createTopics(Arrays.asList( + new NewTopic("foo", 1, 3.toShort), + new NewTopic("bar", 2, 3.toShort))).values() + createResults.get("foo").get() + createResults.get("bar").get() + val increaseResults = admin.createPartitions(Map( + "foo" -> NewPartitions.increaseTo(3), + "bar" -> NewPartitions.increaseTo(2)).asJava).values() + increaseResults.get("foo").get() + assertEquals(classOf[InvalidPartitionsException], assertThrows( + classOf[ExecutionException], () => increaseResults.get("bar").get()).getCause.getClass) + } finally { + admin.close() + } + } finally { + cluster.close() + } + } private def clusterImage( cluster: KafkaClusterTestKit, brokerId: Int diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index 45cd92057f..aabf3f9345 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -124,7 +124,7 @@ class KRaftQuorumImplementation( util.EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR)) val sharedServer = new SharedServer(config, metaPropertiesEnsemble, - Time.SYSTEM, + time, new Metrics(), controllerQuorumVotersFuture, faultHandlerFactory) diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala index e52bc4e567..d442df33ad 100644 --- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala @@ -48,7 +48,7 @@ import org.apache.kafka.raft.RaftConfig import org.apache.kafka.security.PasswordEncoder import org.apache.kafka.server.ControllerRequestCompletionHandler import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion, ProducerIdsBlock} -import org.apache.kafka.server.config.ConfigType +import org.apache.kafka.server.config.{ConfigEntityName, ConfigType} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue, fail} import org.junit.jupiter.api.{Assumptions, Timeout} import org.junit.jupiter.api.extension.ExtendWith @@ -227,15 +227,19 @@ class ZkMigrationIntegrationTest { createTopicResult.all().get(60, TimeUnit.SECONDS) val quotas = new util.ArrayList[ClientQuotaAlteration]() - quotas.add(new ClientQuotaAlteration( - new ClientQuotaEntity(Map("user" -> "user1").asJava), - List(new ClientQuotaAlteration.Op("consumer_byte_rate", 1000.0)).asJava)) - quotas.add(new ClientQuotaAlteration( - new ClientQuotaEntity(Map("user" -> "user1", "client-id" -> "clientA").asJava), + val defaultUserEntity = new ClientQuotaEntity(Map(ClientQuotaEntity.USER -> ConfigEntityName.DEFAULT).asJava) + quotas.add(new ClientQuotaAlteration(defaultUserEntity, List(new ClientQuotaAlteration.Op("consumer_byte_rate", 900.0)).asJava)) + val defaultClientIdEntity = new ClientQuotaEntity(Map(ClientQuotaEntity.CLIENT_ID -> ConfigEntityName.DEFAULT).asJava) + quotas.add(new ClientQuotaAlteration(defaultClientIdEntity, List(new ClientQuotaAlteration.Op("consumer_byte_rate", 900.0)).asJava)) + val defaultIpEntity = new ClientQuotaEntity(Map(ClientQuotaEntity.IP -> null.asInstanceOf[String]).asJava) + quotas.add(new ClientQuotaAlteration(defaultIpEntity, List(new ClientQuotaAlteration.Op("connection_creation_rate", 9.0)).asJava)) + val userEntity = new ClientQuotaEntity(Map(ClientQuotaEntity.USER -> "user/1@prod").asJava) + quotas.add(new ClientQuotaAlteration(userEntity, List(new ClientQuotaAlteration.Op("consumer_byte_rate", 1000.0)).asJava)) + val userClientEntity = new ClientQuotaEntity(Map(ClientQuotaEntity.USER -> "user/1@prod", ClientQuotaEntity.CLIENT_ID -> "client/1@domain").asJava) + quotas.add(new ClientQuotaAlteration(userClientEntity, List(new ClientQuotaAlteration.Op("consumer_byte_rate", 800.0), new ClientQuotaAlteration.Op("producer_byte_rate", 100.0)).asJava)) - quotas.add(new ClientQuotaAlteration( - new ClientQuotaEntity(Map("ip" -> "8.8.8.8").asJava), - List(new ClientQuotaAlteration.Op("connection_creation_rate", 10.0)).asJava)) + val ipEntity = new ClientQuotaEntity(Map(ClientQuotaEntity.IP -> "8.8.8.8").asJava) + quotas.add(new ClientQuotaAlteration(ipEntity, List(new ClientQuotaAlteration.Op("connection_creation_rate", 10.0)).asJava)) admin.alterClientQuotas(quotas).all().get(60, TimeUnit.SECONDS) val zkClient = clusterInstance.asInstanceOf[ZkClusterInstance].getUnderlying().zkClient @@ -271,7 +275,13 @@ class ZkMigrationIntegrationTest { assertEquals(10, image.topics().getTopic("test-topic-3").partitions().size()) val clientQuotas = image.clientQuotas().entities() - assertEquals(3, clientQuotas.size()) + assertEquals(6, clientQuotas.size()) + assertEquals(true, clientQuotas.containsKey(defaultUserEntity)) + assertEquals(true, clientQuotas.containsKey(defaultClientIdEntity)) + assertEquals(true, clientQuotas.containsKey(new ClientQuotaEntity(Map(ClientQuotaEntity.IP -> "").asJava))) // default ip + assertEquals(true, clientQuotas.containsKey(userEntity)) + assertEquals(true, clientQuotas.containsKey(userClientEntity)) + assertEquals(true, clientQuotas.containsKey(ipEntity)) } migrationState = migrationClient.releaseControllerLeadership(migrationState) diff --git a/core/src/test/scala/kafka/log/streamaspect/ElasticLogTest.scala b/core/src/test/scala/kafka/log/streamaspect/ElasticLogTest.scala index 7d722ef5d2..08c968c023 100644 --- a/core/src/test/scala/kafka/log/streamaspect/ElasticLogTest.scala +++ b/core/src/test/scala/kafka/log/streamaspect/ElasticLogTest.scala @@ -95,7 +95,7 @@ class ElasticLogTest { initialOffset: Long = 0L): Unit = { log.append(lastOffset = initialOffset + records.size - 1, largestTimestamp = records.head.timestamp, - shallowOffsetOfMaxTimestamp = initialOffset, + offsetOfMaxTimestamp = initialOffset, records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList: _*)) } diff --git a/core/src/test/scala/kafka/log/streamaspect/ElasticUnifiedLogTest.scala b/core/src/test/scala/kafka/log/streamaspect/ElasticUnifiedLogTest.scala index 1e97730772..2275e739d9 100644 --- a/core/src/test/scala/kafka/log/streamaspect/ElasticUnifiedLogTest.scala +++ b/core/src/test/scala/kafka/log/streamaspect/ElasticUnifiedLogTest.scala @@ -562,4 +562,8 @@ class ElasticUnifiedLogTest extends UnifiedLogTest { } else assertNull(readInfo) } + + override def testFetchLatestTieredTimestampWithRemoteStorage(): Unit = { + // AutoMQ embedded tiered storage in S3Stream + } } diff --git a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala deleted file mode 100644 index 18c7a0a8f8..0000000000 --- a/core/src/test/scala/unit/kafka/admin/ConsumerGroupCommandTest.scala +++ /dev/null @@ -1,225 +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.time.Duration -import java.util.concurrent.{ExecutorService, Executors, TimeUnit} -import java.util.{Collections, Properties} -import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService} -import kafka.api.BaseConsumerTest -import kafka.integration.KafkaServerTestHarness -import kafka.server.KafkaConfig -import kafka.utils.TestUtils -import org.apache.kafka.clients.admin.AdminClientConfig -import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, GroupProtocol, KafkaConsumer, RangeAssignor} -import org.apache.kafka.common.{PartitionInfo, TopicPartition} -import org.apache.kafka.common.errors.WakeupException -import org.apache.kafka.common.serialization.StringDeserializer -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} - -import scala.jdk.CollectionConverters._ -import scala.collection.mutable.ArrayBuffer - -class ConsumerGroupCommandTest extends KafkaServerTestHarness { - import ConsumerGroupCommandTest._ - - val topic = "foo" - val group = "test.group" - - private var consumerGroupService: List[ConsumerGroupService] = List() - private var consumerGroupExecutors: List[AbstractConsumerGroupExecutor] = List() - - // configure the servers and clients - override def generateConfigs = { - val configs = TestUtils.createBrokerConfigs(1, zkConnectOrNull, enableControlledShutdown = false) - - if (isNewGroupCoordinatorEnabled()) { - configs.foreach(_.setProperty(KafkaConfig.NewGroupCoordinatorEnableProp, "true")) - } - - configs.map(KafkaConfig.fromProps) - } - - @BeforeEach - override def setUp(testInfo: TestInfo): Unit = { - super.setUp(testInfo) - createTopic(topic, 1, 1) - } - - @AfterEach - override def tearDown(): Unit = { - consumerGroupService.foreach(_.close()) - consumerGroupExecutors.foreach(_.shutdown()) - super.tearDown() - } - - def committedOffsets(topic: String = topic, group: String = group): collection.Map[TopicPartition, Long] = { - val consumer = createNoAutoCommitConsumer(group) - try { - val partitions: Set[TopicPartition] = consumer.partitionsFor(topic) - .asScala.toSet.map {partitionInfo : PartitionInfo => new TopicPartition(partitionInfo.topic, partitionInfo.partition)} - consumer.committed(partitions.asJava).asScala.filter(_._2 != null).map { case (k, v) => k -> v.offset } - } finally { - consumer.close() - } - } - - def createNoAutoCommitConsumer(group: String): Consumer[String, String] = { - val props = new Properties - props.put("bootstrap.servers", bootstrapServers()) - props.put("group.id", group) - props.put("enable.auto.commit", "false") - new KafkaConsumer(props, new StringDeserializer, new StringDeserializer) - } - - def getConsumerGroupService(args: Array[String]): ConsumerGroupService = { - val opts = new ConsumerGroupCommandOptions(args) - val service = new ConsumerGroupService(opts, Map(AdminClientConfig.RETRIES_CONFIG -> Int.MaxValue.toString)) - consumerGroupService = service :: consumerGroupService - service - } - - def addConsumerGroupExecutor(numConsumers: Int, - topic: String = topic, - group: String = group, - strategy: String = classOf[RangeAssignor].getName, - remoteAssignor: Option[String] = None, - customPropsOpt: Option[Properties] = None, - syncCommit: Boolean = false, - groupProtocol: String = GroupProtocol.CLASSIC.toString): ConsumerGroupExecutor = { - val executor = new ConsumerGroupExecutor(bootstrapServers(), numConsumers, group, groupProtocol, topic, strategy, remoteAssignor, customPropsOpt, syncCommit) - addExecutor(executor) - executor - } - - def addSimpleGroupExecutor(partitions: Iterable[TopicPartition] = Seq(new TopicPartition(topic, 0)), - group: String = group): SimpleConsumerGroupExecutor = { - val executor = new SimpleConsumerGroupExecutor(bootstrapServers(), group, partitions) - addExecutor(executor) - executor - } - - private def addExecutor(executor: AbstractConsumerGroupExecutor): AbstractConsumerGroupExecutor = { - consumerGroupExecutors = executor :: consumerGroupExecutors - executor - } - -} - -object ConsumerGroupCommandTest { - def getTestQuorumAndGroupProtocolParametersAll() = BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll() - - abstract class AbstractConsumerRunnable(broker: String, groupId: String, customPropsOpt: Option[Properties] = None, - syncCommit: Boolean = false) extends Runnable { - val props = new Properties - configure(props) - customPropsOpt.foreach(props.asScala ++= _.asScala) - val consumer = new KafkaConsumer(props) - - def configure(props: Properties): Unit = { - props.put("bootstrap.servers", broker) - props.put("group.id", groupId) - props.put("key.deserializer", classOf[StringDeserializer].getName) - props.put("value.deserializer", classOf[StringDeserializer].getName) - } - - def subscribe(): Unit - - def run(): Unit = { - try { - subscribe() - while (true) { - consumer.poll(Duration.ofMillis(Long.MaxValue)) - if (syncCommit) - consumer.commitSync() - } - } catch { - case _: WakeupException => // OK - } finally { - consumer.close() - } - } - - def shutdown(): Unit = { - consumer.wakeup() - } - } - - class ConsumerRunnable(broker: String, groupId: String, groupProtocol: String, topic: String, strategy: String, - remoteAssignor: Option[String], customPropsOpt: Option[Properties] = None, syncCommit: Boolean = false) - extends AbstractConsumerRunnable(broker, groupId, customPropsOpt, syncCommit) { - - override def configure(props: Properties): Unit = { - super.configure(props) - props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol) - if (groupProtocol.toUpperCase == GroupProtocol.CONSUMER.toString) { - remoteAssignor.foreach { assignor => - props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, assignor) - } - } else { - props.put("partition.assignment.strategy", strategy) - } - } - - override def subscribe(): Unit = { - consumer.subscribe(Collections.singleton(topic)) - } - } - - class SimpleConsumerRunnable(broker: String, groupId: String, partitions: Iterable[TopicPartition]) - extends AbstractConsumerRunnable(broker, groupId) { - - override def subscribe(): Unit = { - consumer.assign(partitions.toList.asJava) - } - } - - class AbstractConsumerGroupExecutor(numThreads: Int) { - private val executor: ExecutorService = Executors.newFixedThreadPool(numThreads) - private val consumers = new ArrayBuffer[AbstractConsumerRunnable]() - - def submit(consumerThread: AbstractConsumerRunnable): Unit = { - consumers += consumerThread - executor.submit(consumerThread) - } - - def shutdown(): Unit = { - consumers.foreach(_.shutdown()) - executor.shutdown() - executor.awaitTermination(5000, TimeUnit.MILLISECONDS) - } - } - - class ConsumerGroupExecutor(broker: String, numConsumers: Int, groupId: String, groupProtocol: String, topic: String, strategy: String, - remoteAssignor: Option[String], customPropsOpt: Option[Properties] = None, syncCommit: Boolean = false) - extends AbstractConsumerGroupExecutor(numConsumers) { - - for (_ <- 1 to numConsumers) { - submit(new ConsumerRunnable(broker, groupId, groupProtocol, topic, strategy, remoteAssignor, customPropsOpt, syncCommit)) - } - - } - - class SimpleConsumerGroupExecutor(broker: String, groupId: String, partitions: Iterable[TopicPartition]) - extends AbstractConsumerGroupExecutor(1) { - - submit(new SimpleConsumerRunnable(broker, groupId, partitions)) - } - -} - diff --git a/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala b/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala deleted file mode 100644 index e98404f496..0000000000 --- a/core/src/test/scala/unit/kafka/admin/DescribeConsumerGroupTest.scala +++ /dev/null @@ -1,747 +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.util.Properties -import kafka.utils.{Exit, TestInfoUtils, TestUtils} -import org.apache.kafka.clients.consumer.{ConsumerConfig, RoundRobinAssignor} -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.errors.TimeoutException -import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{MethodSource, ValueSource} - -import scala.concurrent.ExecutionException -import scala.util.Random - -class DescribeConsumerGroupTest extends ConsumerGroupCommandTest { - private val describeTypeOffsets = Array(Array(""), Array("--offsets")) - private val describeTypeMembers = Array(Array("--members"), Array("--members", "--verbose")) - private val describeTypeState = Array(Array("--state")) - private val describeTypes = describeTypeOffsets ++ describeTypeMembers ++ describeTypeState - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeNonExistingGroup(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - val missingGroup = "missing.group" - - for (describeType <- describeTypes) { - // note the group to be queried is a different (non-existing) group - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", missingGroup) ++ describeType - val service = getConsumerGroupService(cgcArgs) - - val output = TestUtils.grabConsoleOutput(service.describeGroups()) - assertTrue(output.contains(s"Consumer group '$missingGroup' does not exist."), - s"Expected error was not detected for describe option '${describeType.mkString(" ")}'") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeWithMultipleSubActions(quorum: String): Unit = { - var exitStatus: Option[Int] = None - var exitMessage: Option[String] = None - Exit.setExitProcedure { (status, err) => - exitStatus = Some(status) - exitMessage = err - throw new RuntimeException - } - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group, "--members", "--state") - try { - ConsumerGroupCommand.main(cgcArgs) - } catch { - case e: RuntimeException => //expected - } finally { - Exit.resetExitProcedure() - } - assertEquals(Some(1), exitStatus) - assertTrue(exitMessage.get.contains("Option [describe] takes at most one of these options")) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeWithStateValue(quorum: String): Unit = { - var exitStatus: Option[Int] = None - var exitMessage: Option[String] = None - Exit.setExitProcedure { (status, err) => - exitStatus = Some(status) - exitMessage = err - throw new RuntimeException - } - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--all-groups", "--state", "Stable") - try { - ConsumerGroupCommand.main(cgcArgs) - } catch { - case e: RuntimeException => //expected - } finally { - Exit.resetExitProcedure() - } - assertEquals(Some(1), exitStatus) - assertTrue(exitMessage.get.contains("Option [describe] does not take a value for [state]")) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeOffsetsOfNonExistingGroup(quorum: String, groupProtocol: String): Unit = { - val group = "missing.group" - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - // note the group to be queried is a different (non-existing) group - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - val (state, assignments) = service.collectGroupOffsets(group) - assertTrue(state.contains("Dead") && assignments.contains(List()), - s"Expected the state to be 'Dead', with no members in the group '$group'.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeMembersOfNonExistingGroup(quorum: String, groupProtocol: String): Unit = { - val group = "missing.group" - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - // note the group to be queried is a different (non-existing) group - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - val (state, assignments) = service.collectGroupMembers(group, false) - assertTrue(state.contains("Dead") && assignments.contains(List()), - s"Expected the state to be 'Dead', with no members in the group '$group'.") - - val (state2, assignments2) = service.collectGroupMembers(group, true) - assertTrue(state2.contains("Dead") && assignments2.contains(List()), - s"Expected the state to be 'Dead', with no members in the group '$group' (verbose option).") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeStateOfNonExistingGroup(quorum: String, groupProtocol: String): Unit = { - val group = "missing.group" - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - // note the group to be queried is a different (non-existing) group - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - val state = service.collectGroupState(group) - assertTrue(state.state == "Dead" && state.numMembers == 0 && - state.coordinator != null && brokers.map(_.config.brokerId).toList.contains(state.coordinator.id), - s"Expected the state to be 'Dead', with no members in the group '$group'." - ) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeExistingGroup(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - for (describeType <- describeTypes) { - val group = this.group + describeType.mkString("") - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, group = group, groupProtocol = groupProtocol) - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) ++ describeType - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (output, error) = TestUtils.grabConsoleOutputAndError(service.describeGroups()) - output.trim.split("\n").length == 2 && error.isEmpty - }, s"Expected a data row and no error in describe results with describe type ${describeType.mkString(" ")}.") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeExistingGroups(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // Create N single-threaded consumer groups from a single-partition topic - val groups = (for (describeType <- describeTypes) yield { - val group = this.group + describeType.mkString("") - addConsumerGroupExecutor(numConsumers = 1, group = group, groupProtocol = groupProtocol) - Array("--group", group) - }).flatten - - val expectedNumLines = describeTypes.length * 2 - - for (describeType <- describeTypes) { - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe") ++ groups ++ describeType - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (output, error) = TestUtils.grabConsoleOutputAndError(service.describeGroups()) - val numLines = output.trim.split("\n").count(line => line.nonEmpty) - (numLines == expectedNumLines) && error.isEmpty - }, s"Expected a data row and no error in describe results with describe type ${describeType.mkString(" ")}.") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeAllExistingGroups(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // Create N single-threaded consumer groups from a single-partition topic - for (describeType <- describeTypes) { - val group = this.group + describeType.mkString("") - addConsumerGroupExecutor(numConsumers = 1, group = group, groupProtocol = groupProtocol) - } - - val expectedNumLines = describeTypes.length * 2 - - for (describeType <- describeTypes) { - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--all-groups") ++ describeType - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (output, error) = TestUtils.grabConsoleOutputAndError(service.describeGroups()) - val numLines = output.trim.split("\n").count(line => line.nonEmpty) - (numLines == expectedNumLines) && error.isEmpty - }, s"Expected a data row and no error in describe results with describe type ${describeType.mkString(" ")}.") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeOffsetsOfExistingGroup(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupOffsets(group) - state.contains("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 1 && - assignments.get.filter(_.group == group).head.consumerId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) && - assignments.get.filter(_.group == group).head.clientId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) && - assignments.get.filter(_.group == group).head.host.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) - }, s"Expected a 'Stable' group status, rows and valid values for consumer id / client id / host columns in describe results for group $group.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeMembersOfExistingGroup(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupMembers(group, false) - state.contains("Stable") && - (assignments match { - case Some(memberAssignments) => - memberAssignments.count(_.group == group) == 1 && - memberAssignments.filter(_.group == group).head.consumerId != ConsumerGroupCommand.MISSING_COLUMN_VALUE && - memberAssignments.filter(_.group == group).head.clientId != ConsumerGroupCommand.MISSING_COLUMN_VALUE && - memberAssignments.filter(_.group == group).head.host != ConsumerGroupCommand.MISSING_COLUMN_VALUE - case None => - false - }) - }, s"Expected a 'Stable' group status, rows and valid member information for group $group.") - - val (_, assignments) = service.collectGroupMembers(group, true) - assignments match { - case None => - fail(s"Expected partition assignments for members of group $group") - case Some(memberAssignments) => - assertTrue(memberAssignments.size == 1 && memberAssignments.head.assignment.size == 1, - s"Expected a topic partition assigned to the single group member for group $group") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeStateOfExistingGroup(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor( - numConsumers = 1, - groupProtocol = groupProtocol, - // This is only effective when new protocol is used. - remoteAssignor = Some("range") - ) - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val state = service.collectGroupState(group) - state.state == "Stable" && - state.numMembers == 1 && - state.assignmentStrategy == "range" && - state.coordinator != null && - brokers.map(_.config.brokerId).toList.contains(state.coordinator.id) - }, s"Expected a 'Stable' group status, with one member and round robin assignment strategy for group $group.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeStateOfExistingGroupWithNonDefaultAssignor(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - val expectedName = if (groupProtocol == "consumer") { - addConsumerGroupExecutor(numConsumers = 1, remoteAssignor = Some("range"), groupProtocol = groupProtocol) - "range" - } else { - addConsumerGroupExecutor(numConsumers = 1, strategy = classOf[RoundRobinAssignor].getName, groupProtocol = groupProtocol) - "roundrobin" - } - - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val state = service.collectGroupState(group) - state.state == "Stable" && - state.numMembers == 1 && - state.assignmentStrategy == expectedName && - state.coordinator != null && - brokers.map(_.config.brokerId).toList.contains(state.coordinator.id) - }, s"Expected a 'Stable' group status, with one member and $expectedName assignment strategy for group $group.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeExistingGroupWithNoMembers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - for (describeType <- describeTypes) { - val group = this.group + describeType.mkString("") - // run one consumer in the group consuming from a single-partition topic - val executor = addConsumerGroupExecutor(numConsumers = 1, group = group, groupProtocol = groupProtocol) - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) ++ describeType - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (output, error) = TestUtils.grabConsoleOutputAndError(service.describeGroups()) - output.trim.split("\n").length == 2 && error.isEmpty - }, s"Expected describe group results with one data row for describe type '${describeType.mkString(" ")}'") - - // stop the consumer so the group has no active member anymore - executor.shutdown() - TestUtils.waitUntilTrue(() => { - TestUtils.grabConsoleError(service.describeGroups()).contains(s"Consumer group '$group' has no active members.") - }, s"Expected no active member in describe group results with describe type ${describeType.mkString(" ")}") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeOffsetsOfExistingGroupWithNoMembers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - val executor = addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol, syncCommit = true) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupOffsets(group) - state.contains("Stable") && assignments.exists(_.exists(assignment => assignment.group == group && assignment.offset.isDefined)) - }, "Expected the group to initially become stable, and to find group in assignments after initial offset commit.") - - // stop the consumer so the group has no active member anymore - executor.shutdown() - - val (result, succeeded) = TestUtils.computeUntilTrue(service.collectGroupOffsets(group)) { - case (state, assignments) => - val testGroupAssignments = assignments.toSeq.flatMap(_.filter(_.group == group)) - def assignment = testGroupAssignments.head - state.contains("Empty") && - testGroupAssignments.size == 1 && - assignment.consumerId.exists(_.trim == ConsumerGroupCommand.MISSING_COLUMN_VALUE) && // the member should be gone - assignment.clientId.exists(_.trim == ConsumerGroupCommand.MISSING_COLUMN_VALUE) && - assignment.host.exists(_.trim == ConsumerGroupCommand.MISSING_COLUMN_VALUE) - } - val (state, assignments) = result - assertTrue(succeeded, s"Expected no active member in describe group results, state: $state, assignments: $assignments") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeMembersOfExistingGroupWithNoMembers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - val executor = addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupMembers(group, false) - state.contains("Stable") && assignments.exists(_.exists(_.group == group)) - }, "Expected the group to initially become stable, and to find group in assignments after initial offset commit.") - - // stop the consumer so the group has no active member anymore - executor.shutdown() - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupMembers(group, false) - state.contains("Empty") && assignments.isDefined && assignments.get.isEmpty - }, s"Expected no member in describe group members results for group '$group'") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeStateOfExistingGroupWithNoMembers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run one consumer in the group consuming from a single-partition topic - val executor = addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val state = service.collectGroupState(group) - state.state == "Stable" && - state.numMembers == 1 && - state.coordinator != null && - brokers.map(_.config.brokerId).toList.contains(state.coordinator.id) - }, s"Expected the group '$group' to initially become stable, and have a single member.") - - // stop the consumer so the group has no active member anymore - executor.shutdown() - - TestUtils.waitUntilTrue(() => { - val state = service.collectGroupState(group) - state.state == "Empty" && state.numMembers == 0 - }, s"Expected the group '$group' to become empty after the only member leaving.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeWithConsumersWithoutAssignedPartitions(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - for (describeType <- describeTypes) { - val group = this.group + describeType.mkString("") - // run two consumers in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 2, group = group, groupProtocol = groupProtocol) - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) ++ describeType - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (output, error) = TestUtils.grabConsoleOutputAndError(service.describeGroups()) - val expectedNumRows = if (describeTypeMembers.contains(describeType)) 3 else 2 - error.isEmpty && output.trim.split("\n").size == expectedNumRows - }, s"Expected a single data row in describe group result with describe type '${describeType.mkString(" ")}'") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeOffsetsWithConsumersWithoutAssignedPartitions(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run two consumers in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 2, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupOffsets(group) - state.contains("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 1 && - assignments.get.count { x => x.group == group && x.partition.isDefined } == 1 - }, "Expected rows for consumers with no assigned partitions in describe group results") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeMembersWithConsumersWithoutAssignedPartitions(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run two consumers in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 2, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupMembers(group, false) - state.contains("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 2 && - assignments.get.count { x => x.group == group && x.numPartitions == 1 } == 1 && - assignments.get.count { x => x.group == group && x.numPartitions == 0 } == 1 && - !assignments.get.exists(_.assignment.nonEmpty) - }, "Expected rows for consumers with no assigned partitions in describe group results") - - val (state, assignments) = service.collectGroupMembers(group, true) - assertTrue(state.contains("Stable") && assignments.get.count(_.assignment.nonEmpty) > 0, - "Expected additional columns in verbose version of describe members") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeStateWithConsumersWithoutAssignedPartitions(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - // run two consumers in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 2, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val state = service.collectGroupState(group) - state.state == "Stable" && state.numMembers == 2 - }, "Expected two consumers in describe group results") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeWithMultiPartitionTopicAndMultipleConsumers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - val topic2 = "foo2" - createTopic(topic2, 2, 1) - - for (describeType <- describeTypes) { - val group = this.group + describeType.mkString("") - // run two consumers in the group consuming from a two-partition topic - addConsumerGroupExecutor(2, topic2, group = group, groupProtocol = groupProtocol) - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) ++ describeType - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (output, error) = TestUtils.grabConsoleOutputAndError(service.describeGroups()) - val expectedNumRows = if (describeTypeState.contains(describeType)) 2 else 3 - error.isEmpty && output.trim.split("\n").size == expectedNumRows - }, s"Expected a single data row in describe group result with describe type '${describeType.mkString(" ")}'") - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeOffsetsWithMultiPartitionTopicAndMultipleConsumers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - val topic2 = "foo2" - createTopic(topic2, 2, 1) - - // run two consumers in the group consuming from a two-partition topic - addConsumerGroupExecutor(numConsumers = 2, topic2, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupOffsets(group) - state.contains("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 2 && - assignments.get.count { x => x.group == group && x.partition.isDefined } == 2 && - assignments.get.count { x => x.group == group && x.partition.isEmpty } == 0 - }, "Expected two rows (one row per consumer) in describe group results.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeMembersWithMultiPartitionTopicAndMultipleConsumers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - val topic2 = "foo2" - createTopic(topic2, 2, 1) - - // run two consumers in the group consuming from a two-partition topic - addConsumerGroupExecutor(numConsumers = 2, topic2, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupMembers(group, false) - state.contains("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 2 && - assignments.get.count { x => x.group == group && x.numPartitions == 1 } == 2 && - assignments.get.count { x => x.group == group && x.numPartitions == 0 } == 0 - }, "Expected two rows (one row per consumer) in describe group members results.") - - val (state, assignments) = service.collectGroupMembers(group, true) - assertTrue(state.contains("Stable") && assignments.get.count(_.assignment.isEmpty) == 0, - "Expected additional columns in verbose version of describe members") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeStateWithMultiPartitionTopicAndMultipleConsumers(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - val topic2 = "foo2" - createTopic(topic2, 2, 1) - - // run two consumers in the group consuming from a two-partition topic - addConsumerGroupExecutor(numConsumers = 2, topic2, groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val state = service.collectGroupState(group) - state.state == "Stable" && state.group == group && state.numMembers == 2 - }, "Expected a stable group with two members in describe group state result.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft", "kraft+kip848")) - def testDescribeSimpleConsumerGroup(quorum: String): Unit = { - // Ensure that the offsets of consumers which don't use group management are still displayed - - createOffsetsTopic() - val topic2 = "foo2" - createTopic(topic2, 2, 1) - addSimpleGroupExecutor(Seq(new TopicPartition(topic2, 0), new TopicPartition(topic2, 1))) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupOffsets(group) - state.contains("Empty") && assignments.isDefined && assignments.get.count(_.group == group) == 2 - }, "Expected a stable group with two members in describe group state result.") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeGroupWithShortInitializationTimeout(quorum: String, groupProtocol: String): Unit = { - // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't - // complete before the timeout expires - - val describeType = describeTypes(Random.nextInt(describeTypes.length)) - val group = this.group + describeType.mkString("") - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - // set the group initialization timeout too low for the group to stabilize - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--timeout", "1", "--group", group) ++ describeType - val service = getConsumerGroupService(cgcArgs) - - val e = assertThrows(classOf[ExecutionException], () => TestUtils.grabConsoleOutputAndError(service.describeGroups())) - assertEquals(classOf[TimeoutException], e.getCause.getClass) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeGroupOffsetsWithShortInitializationTimeout(quorum: String, groupProtocol: String): Unit = { - // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't - // complete before the timeout expires - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - - // set the group initialization timeout too low for the group to stabilize - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group, "--timeout", "1") - val service = getConsumerGroupService(cgcArgs) - - val e = assertThrows(classOf[ExecutionException], () => service.collectGroupOffsets(group)) - assertEquals(classOf[TimeoutException], e.getCause.getClass) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeGroupMembersWithShortInitializationTimeout(quorum: String, groupProtocol: String): Unit = { - // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't - // complete before the timeout expires - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - - // set the group initialization timeout too low for the group to stabilize - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group, "--timeout", "1") - val service = getConsumerGroupService(cgcArgs) - - var e = assertThrows(classOf[ExecutionException], () => service.collectGroupMembers(group, false)) - assertEquals(classOf[TimeoutException], e.getCause.getClass) - e = assertThrows(classOf[ExecutionException], () => service.collectGroupMembers(group, true)) - assertEquals(classOf[TimeoutException], e.getCause.getClass) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeGroupStateWithShortInitializationTimeout(quorum: String, groupProtocol: String): Unit = { - // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't - // complete before the timeout expires - - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, groupProtocol = groupProtocol) - - // set the group initialization timeout too low for the group to stabilize - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group, "--timeout", "1") - val service = getConsumerGroupService(cgcArgs) - - val e = assertThrows(classOf[ExecutionException], () => service.collectGroupState(group)) - assertEquals(classOf[TimeoutException], e.getCause.getClass) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeWithUnrecognizedNewConsumerOption(quorum: String): Unit = { - val cgcArgs = Array("--new-consumer", "--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - assertThrows(classOf[joptsimple.OptionException], () => getConsumerGroupService(cgcArgs)) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeNonOffsetCommitGroup(quorum: String, groupProtocol: String): Unit = { - createOffsetsTopic() - - val customProps = new Properties - // create a consumer group that never commits offsets - customProps.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") - // run one consumer in the group consuming from a single-partition topic - addConsumerGroupExecutor(numConsumers = 1, customPropsOpt = Some(customProps), groupProtocol = groupProtocol) - - val cgcArgs = Array("--bootstrap-server", bootstrapServers(), "--describe", "--group", group) - val service = getConsumerGroupService(cgcArgs) - - TestUtils.waitUntilTrue(() => { - val (state, assignments) = service.collectGroupOffsets(group) - state.contains("Stable") && - assignments.isDefined && - assignments.get.count(_.group == group) == 1 && - assignments.get.filter(_.group == group).head.consumerId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) && - assignments.get.filter(_.group == group).head.clientId.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) && - assignments.get.filter(_.group == group).head.host.exists(_.trim != ConsumerGroupCommand.MISSING_COLUMN_VALUE) - }, s"Expected a 'Stable' group status, rows and valid values for consumer id / client id / host columns in describe results for non-offset-committing group $group.") - } - -} - diff --git a/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala b/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala deleted file mode 100644 index 4aafedf0ab..0000000000 --- a/core/src/test/scala/unit/kafka/admin/ResetConsumerGroupOffsetTest.scala +++ /dev/null @@ -1,514 +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.io.{BufferedWriter, FileWriter} -import java.text.{SimpleDateFormat} -import java.util.{Calendar, Date, Properties} - -import joptsimple.OptionException -import kafka.admin.ConsumerGroupCommand.ConsumerGroupService -import kafka.server.KafkaConfig -import kafka.utils.TestUtils -import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.test -import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test - -import scala.jdk.CollectionConverters._ -import scala.collection.Seq - - - -/** - * Test cases by: - * - Non-existing consumer group - * - One for each scenario, with scope=all-topics - * - scope=one topic, scenario=to-earliest - * - scope=one topic+partitions, scenario=to-earliest - * - scope=topics, scenario=to-earliest - * - scope=topics+partitions, scenario=to-earliest - * - export/import - */ -class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { - - val overridingProps = new Properties() - val topic1 = "foo1" - val topic2 = "foo2" - - override def generateConfigs: Seq[KafkaConfig] = { - TestUtils.createBrokerConfigs(1, zkConnect, enableControlledShutdown = false) - .map(KafkaConfig.fromProps(_, overridingProps)) - } - - private def basicArgs: Array[String] = { - Array("--reset-offsets", - "--bootstrap-server", bootstrapServers(), - "--timeout", test.TestUtils.DEFAULT_MAX_WAIT_MS.toString) - } - - private def buildArgsForGroups(groups: Seq[String], args: String*): Array[String] = { - val groupArgs = groups.flatMap(group => Seq("--group", group)).toArray - basicArgs ++ groupArgs ++ args - } - - private def buildArgsForGroup(group: String, args: String*): Array[String] = { - buildArgsForGroups(Seq(group), args: _*) - } - - private def buildArgsForAllGroups(args: String*): Array[String] = { - basicArgs ++ Array("--all-groups") ++ args - } - - @Test - def testResetOffsetsNotExistingGroup(): Unit = { - val group = "missing.group" - val args = buildArgsForGroup(group, "--all-topics", "--to-current", "--execute") - val consumerGroupCommand = getConsumerGroupService(args) - // Make sure we got a coordinator - TestUtils.waitUntilTrue(() => { - consumerGroupCommand.collectGroupState(group).coordinator.host() == "localhost" - }, "Can't find a coordinator") - val resetOffsets = consumerGroupCommand.resetOffsets()(group) - assertEquals(Map.empty, resetOffsets) - assertEquals(resetOffsets, committedOffsets(group = group)) - } - - @Test - def testResetOffsetsExistingTopic(): Unit = { - val group = "new.group" - val args = buildArgsForGroup(group, "--topic", topic, "--to-offset", "50") - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 50, dryRun = true) - resetAndAssertOffsets(args ++ Array("--dry-run"), expectedOffset = 50, dryRun = true) - resetAndAssertOffsets(args ++ Array("--execute"), expectedOffset = 50) - } - - @Test - def testResetOffsetsExistingTopicSelectedGroups(): Unit = { - produceMessages(topic, 100) - val groups = - for (id <- 1 to 3) yield { - val group = this.group + id - val executor = addConsumerGroupExecutor(numConsumers = 1, topic = topic, group = group) - awaitConsumerProgress(count = 100L, group = group) - executor.shutdown() - group - } - val args = buildArgsForGroups(groups,"--topic", topic, "--to-offset", "50") - resetAndAssertOffsets(args, expectedOffset = 50, dryRun = true) - resetAndAssertOffsets(args ++ Array("--dry-run"), expectedOffset = 50, dryRun = true) - resetAndAssertOffsets(args ++ Array("--execute"), expectedOffset = 50) - } - - @Test - def testResetOffsetsExistingTopicAllGroups(): Unit = { - val args = buildArgsForAllGroups("--topic", topic, "--to-offset", "50") - produceMessages(topic, 100) - for (group <- 1 to 3 map (group + _)) { - val executor = addConsumerGroupExecutor(numConsumers = 1, topic = topic, group = group) - awaitConsumerProgress(count = 100L, group = group) - executor.shutdown() - } - resetAndAssertOffsets(args, expectedOffset = 50, dryRun = true) - resetAndAssertOffsets(args ++ Array("--dry-run"), expectedOffset = 50, dryRun = true) - resetAndAssertOffsets(args ++ Array("--execute"), expectedOffset = 50) - } - - @Test - def testResetOffsetsAllTopicsAllGroups(): Unit = { - val args = buildArgsForAllGroups("--all-topics", "--to-offset", "50") - val topics = 1 to 3 map (topic + _) - val groups = 1 to 3 map (group + _) - topics foreach (topic => produceMessages(topic, 100)) - for { - topic <- topics - group <- groups - } { - val executor = addConsumerGroupExecutor(numConsumers = 3, topic = topic, group = group) - awaitConsumerProgress(topic = topic, count = 100L, group = group) - executor.shutdown() - } - resetAndAssertOffsets(args, expectedOffset = 50, dryRun = true, topics = topics) - resetAndAssertOffsets(args ++ Array("--dry-run"), expectedOffset = 50, dryRun = true, topics = topics) - resetAndAssertOffsets(args ++ Array("--execute"), expectedOffset = 50, topics = topics) - } - - @Test - def testResetOffsetsToLocalDateTime(): Unit = { - val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS") - val calendar = Calendar.getInstance() - calendar.add(Calendar.DATE, -1) - - produceMessages(topic, 100) - - val executor = addConsumerGroupExecutor(numConsumers = 1, topic) - awaitConsumerProgress(count = 100L) - executor.shutdown() - - val args = buildArgsForGroup(group, "--all-topics", "--to-datetime", format.format(calendar.getTime), "--execute") - resetAndAssertOffsets(args, expectedOffset = 0) - } - - @Test - def testResetOffsetsToZonedDateTime(): Unit = { - val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX") - - produceMessages(topic, 50) - val checkpoint = new Date() - produceMessages(topic, 50) - - val executor = addConsumerGroupExecutor(numConsumers = 1, topic) - awaitConsumerProgress(count = 100L) - executor.shutdown() - - val args = buildArgsForGroup(group, "--all-topics", "--to-datetime", format.format(checkpoint), "--execute") - resetAndAssertOffsets(args, expectedOffset = 50) - } - - @Test - def testResetOffsetsByDuration(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--by-duration", "PT1M", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - resetAndAssertOffsets(args, expectedOffset = 0) - } - - @Test - def testResetOffsetsByDurationToEarliest(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--by-duration", "PT0.1S", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - resetAndAssertOffsets(args, expectedOffset = 100) - } - - @Test - def testResetOffsetsByDurationFallbackToLatestWhenNoRecords(): Unit = { - val topic = "foo2" - val args = buildArgsForGroup(group, "--topic", topic, "--by-duration", "PT1M", "--execute") - createTopic(topic) - resetAndAssertOffsets(args, expectedOffset = 0, topics = Seq("foo2")) - - adminZkClient.deleteTopic(topic) - } - - @Test - def testResetOffsetsToEarliest(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--to-earliest", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - resetAndAssertOffsets(args, expectedOffset = 0) - } - - @Test - def testResetOffsetsToLatest(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--to-latest", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 200) - } - - @Test - def testResetOffsetsToCurrentOffset(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--to-current", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 100) - } - - @Test - def testResetOffsetsToSpecificOffset(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--to-offset", "1", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - resetAndAssertOffsets(args, expectedOffset = 1) - } - - @Test - def testResetOffsetsShiftPlus(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--shift-by", "50", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 150) - } - - @Test - def testResetOffsetsShiftMinus(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--shift-by", "-50", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 50) - } - - @Test - def testResetOffsetsShiftByLowerThanEarliest(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--shift-by", "-150", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 0) - } - - @Test - def testResetOffsetsShiftByHigherThanLatest(): Unit = { - val args = buildArgsForGroup(group, "--all-topics", "--shift-by", "150", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - produceMessages(topic, 100) - resetAndAssertOffsets(args, expectedOffset = 200) - } - - @Test - def testResetOffsetsToEarliestOnOneTopic(): Unit = { - val args = buildArgsForGroup(group, "--topic", topic, "--to-earliest", "--execute") - produceConsumeAndShutdown(topic, group, totalMessages = 100) - resetAndAssertOffsets(args, expectedOffset = 0) - } - - @Test - def testResetOffsetsToEarliestOnOneTopicAndPartition(): Unit = { - val topic = "bar" - createTopic(topic, 2, 1) - - val args = buildArgsForGroup(group, "--topic", s"$topic:1", "--to-earliest", "--execute") - val consumerGroupCommand = getConsumerGroupService(args) - - produceConsumeAndShutdown(topic, group, totalMessages = 100, numConsumers = 2) - val priorCommittedOffsets = committedOffsets(topic = topic) - - val tp0 = new TopicPartition(topic, 0) - val tp1 = new TopicPartition(topic, 1) - val expectedOffsets = Map(tp0 -> priorCommittedOffsets(tp0), tp1 -> 0L) - resetAndAssertOffsetsCommitted(consumerGroupCommand, expectedOffsets, topic) - - adminZkClient.deleteTopic(topic) - } - - @Test - def testResetOffsetsToEarliestOnTopics(): Unit = { - val topic1 = "topic1" - val topic2 = "topic2" - createTopic(topic1, 1, 1) - createTopic(topic2, 1, 1) - - val args = buildArgsForGroup(group, "--topic", topic1, "--topic", topic2, "--to-earliest", "--execute") - val consumerGroupCommand = getConsumerGroupService(args) - - produceConsumeAndShutdown(topic1, group, 100, 1) - produceConsumeAndShutdown(topic2, group, 100, 1) - - val tp1 = new TopicPartition(topic1, 0) - val tp2 = new TopicPartition(topic2, 0) - - val allResetOffsets = resetOffsets(consumerGroupCommand)(group).map { case (k, v) => k -> v.offset } - assertEquals(Map(tp1 -> 0L, tp2 -> 0L), allResetOffsets) - assertEquals(Map(tp1 -> 0L), committedOffsets(topic1)) - assertEquals(Map(tp2 -> 0L), committedOffsets(topic2)) - - adminZkClient.deleteTopic(topic1) - adminZkClient.deleteTopic(topic2) - } - - @Test - def testResetOffsetsToEarliestOnTopicsAndPartitions(): Unit = { - val topic1 = "topic1" - val topic2 = "topic2" - - createTopic(topic1, 2, 1) - createTopic(topic2, 2, 1) - - val args = buildArgsForGroup(group, "--topic", s"$topic1:1", "--topic", s"$topic2:1", "--to-earliest", "--execute") - val consumerGroupCommand = getConsumerGroupService(args) - - produceConsumeAndShutdown(topic1, group, 100, 2) - produceConsumeAndShutdown(topic2, group, 100, 2) - - val priorCommittedOffsets1 = committedOffsets(topic1) - val priorCommittedOffsets2 = committedOffsets(topic2) - - val tp1 = new TopicPartition(topic1, 1) - val tp2 = new TopicPartition(topic2, 1) - val allResetOffsets = resetOffsets(consumerGroupCommand)(group).map { case (k, v) => k -> v.offset } - assertEquals(Map(tp1 -> 0, tp2 -> 0), allResetOffsets) - - assertEquals(priorCommittedOffsets1.toMap + (tp1 -> 0L), committedOffsets(topic1)) - assertEquals(priorCommittedOffsets2.toMap + (tp2 -> 0L), committedOffsets(topic2)) - - adminZkClient.deleteTopic(topic1) - adminZkClient.deleteTopic(topic2) - } - - @Test - // This one deals with old CSV export/import format for a single --group arg: "topic,partition,offset" to support old behavior - def testResetOffsetsExportImportPlanSingleGroupArg(): Unit = { - val topic = "bar" - val tp0 = new TopicPartition(topic, 0) - val tp1 = new TopicPartition(topic, 1) - createTopic(topic, 2, 1) - - val cgcArgs = buildArgsForGroup(group, "--all-topics", "--to-offset", "2", "--export") - val consumerGroupCommand = getConsumerGroupService(cgcArgs) - - produceConsumeAndShutdown(topic = topic, group = group, totalMessages = 100, numConsumers = 2) - - val file = TestUtils.tempFile("reset", ".csv") - - val exportedOffsets = consumerGroupCommand.resetOffsets() - val bw = new BufferedWriter(new FileWriter(file)) - bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets)) - bw.close() - assertEquals(Map(tp0 -> 2L, tp1 -> 2L), exportedOffsets(group).map { case (k, v) => k -> v.offset }) - - val cgcArgsExec = buildArgsForGroup(group, "--all-topics", "--from-file", file.getCanonicalPath, "--dry-run") - val consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec) - val importedOffsets = consumerGroupCommandExec.resetOffsets() - assertEquals(Map(tp0 -> 2L, tp1 -> 2L), importedOffsets(group).map { case (k, v) => k -> v.offset }) - - adminZkClient.deleteTopic(topic) - } - - @Test - // This one deals with universal CSV export/import file format "group,topic,partition,offset", - // supporting multiple --group args or --all-groups arg - def testResetOffsetsExportImportPlan(): Unit = { - val group1 = group + "1" - val group2 = group + "2" - val topic1 = "bar1" - val topic2 = "bar2" - val t1p0 = new TopicPartition(topic1, 0) - val t1p1 = new TopicPartition(topic1, 1) - val t2p0 = new TopicPartition(topic2, 0) - val t2p1 = new TopicPartition(topic2, 1) - createTopic(topic1, 2, 1) - createTopic(topic2, 2, 1) - - val cgcArgs = buildArgsForGroups(Seq(group1, group2), "--all-topics", "--to-offset", "2", "--export") - val consumerGroupCommand = getConsumerGroupService(cgcArgs) - - produceConsumeAndShutdown(topic = topic1, group = group1, totalMessages = 100) - produceConsumeAndShutdown(topic = topic2, group = group2, totalMessages = 100) - - awaitConsumerGroupInactive(consumerGroupCommand, group1) - awaitConsumerGroupInactive(consumerGroupCommand, group2) - - val file = TestUtils.tempFile("reset", ".csv") - - val exportedOffsets = consumerGroupCommand.resetOffsets() - val bw = new BufferedWriter(new FileWriter(file)) - bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets)) - bw.close() - assertEquals(Map(t1p0 -> 2L, t1p1 -> 2L), exportedOffsets(group1).map { case (k, v) => k -> v.offset }) - assertEquals(Map(t2p0 -> 2L, t2p1 -> 2L), exportedOffsets(group2).map { case (k, v) => k -> v.offset }) - - // Multiple --group's offset import - val cgcArgsExec = buildArgsForGroups(Seq(group1, group2), "--all-topics", "--from-file", file.getCanonicalPath, "--dry-run") - val consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec) - val importedOffsets = consumerGroupCommandExec.resetOffsets() - assertEquals(Map(t1p0 -> 2L, t1p1 -> 2L), importedOffsets(group1).map { case (k, v) => k -> v.offset }) - assertEquals(Map(t2p0 -> 2L, t2p1 -> 2L), importedOffsets(group2).map { case (k, v) => k -> v.offset }) - - // Single --group offset import using "group,topic,partition,offset" csv format - val cgcArgsExec2 = buildArgsForGroup(group1, "--all-topics", "--from-file", file.getCanonicalPath, "--dry-run") - val consumerGroupCommandExec2 = getConsumerGroupService(cgcArgsExec2) - val importedOffsets2 = consumerGroupCommandExec2.resetOffsets() - assertEquals(Map(t1p0 -> 2L, t1p1 -> 2L), importedOffsets2(group1).map { case (k, v) => k -> v.offset }) - - adminZkClient.deleteTopic(topic) - } - - @Test - def testResetWithUnrecognizedNewConsumerOption(): Unit = { - val cgcArgs = Array("--new-consumer", "--bootstrap-server", bootstrapServers(), "--reset-offsets", "--group", group, "--all-topics", - "--to-offset", "2", "--export") - assertThrows(classOf[OptionException], () => getConsumerGroupService(cgcArgs)) - } - - private def produceMessages(topic: String, numMessages: Int): Unit = { - val records = (0 until numMessages).map(_ => new ProducerRecord[Array[Byte], Array[Byte]](topic, - new Array[Byte](100 * 1000))) - TestUtils.produceMessages(servers, records, acks = 1) - } - - private def produceConsumeAndShutdown(topic: String, group: String, totalMessages: Int, numConsumers: Int = 1): Unit = { - produceMessages(topic, totalMessages) - val executor = addConsumerGroupExecutor(numConsumers = numConsumers, topic = topic, group = group) - awaitConsumerProgress(topic, group, totalMessages) - executor.shutdown() - } - - private def awaitConsumerProgress(topic: String = topic, - group: String = group, - count: Long): Unit = { - val consumer = createNoAutoCommitConsumer(group) - try { - val partitions = consumer.partitionsFor(topic).asScala.map { partitionInfo => - new TopicPartition(partitionInfo.topic, partitionInfo.partition) - }.toSet - - TestUtils.waitUntilTrue(() => { - val committed = consumer.committed(partitions.asJava).values.asScala - val total = committed.foldLeft(0L) { case (currentSum, offsetAndMetadata) => - currentSum + Option(offsetAndMetadata).map(_.offset).getOrElse(0L) - } - total == count - }, "Expected that consumer group has consumed all messages from topic/partition. " + - s"Expected offset: $count. Actual offset: ${committedOffsets(topic, group).values.sum}") - - } finally { - consumer.close() - } - - } - - private def awaitConsumerGroupInactive(consumerGroupService: ConsumerGroupService, group: String): Unit = { - TestUtils.waitUntilTrue(() => { - val state = consumerGroupService.collectGroupState(group).state - state == "Empty" || state == "Dead" - }, s"Expected that consumer group is inactive. Actual state: ${consumerGroupService.collectGroupState(group).state}") - } - - private def resetAndAssertOffsets(args: Array[String], - expectedOffset: Long, - dryRun: Boolean = false, - topics: Seq[String] = Seq(topic)): Unit = { - val consumerGroupCommand = getConsumerGroupService(args) - val expectedOffsets = topics.map(topic => topic -> Map(new TopicPartition(topic, 0) -> expectedOffset)).toMap - val resetOffsetsResultByGroup = resetOffsets(consumerGroupCommand) - - try { - for { - topic <- topics - (group, partitionInfo) <- resetOffsetsResultByGroup - } { - val priorOffsets = committedOffsets(topic = topic, group = group) - assertEquals(expectedOffsets(topic), - partitionInfo.filter(partitionInfo => partitionInfo._1.topic() == topic).map { case (k, v) => k -> v.offset }) - assertEquals(if (dryRun) priorOffsets else expectedOffsets(topic), committedOffsets(topic = topic, group = group)) - } - } finally { - consumerGroupCommand.close() - } - } - - private def resetAndAssertOffsetsCommitted(consumerGroupService: ConsumerGroupService, - expectedOffsets: Map[TopicPartition, Long], - topic: String): Unit = { - val allResetOffsets = resetOffsets(consumerGroupService) - for { - (group, offsetsInfo) <- allResetOffsets - (tp, offsetMetadata) <- offsetsInfo - } { - assertEquals(offsetMetadata.offset(), expectedOffsets(tp)) - assertEquals(expectedOffsets, committedOffsets(topic, group)) - } - } - - private def resetOffsets(consumerGroupService: ConsumerGroupService) = { - consumerGroupService.resetOffsets() - } -} diff --git a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala index 533f67aa75..956db24c01 100644 --- a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala @@ -200,7 +200,8 @@ object AbstractCoordinatorConcurrencyTest { producerId: Long, producerEpoch: Short, baseSequence: Int, - callback: ((Errors, VerificationGuard)) => Unit + callback: ((Errors, VerificationGuard)) => Unit, + supportedOperation: SupportedOperation ): Unit = { // Skip verification callback((Errors.NONE, VerificationGuard.SENTINEL)) diff --git a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala index 9e9ba74260..7a3c5f194f 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala @@ -21,7 +21,7 @@ import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.{NotLeaderOrFollowerException, RecordTooLargeException} -import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{CompressionType, ControlRecordType, MemoryRecords, RecordBatch} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.TransactionResult @@ -343,7 +343,8 @@ class CoordinatorPartitionWriterTest { ArgumentMatchers.eq(10L), ArgumentMatchers.eq(5.toShort), ArgumentMatchers.eq(RecordBatch.NO_SEQUENCE), - callbackCapture.capture() + callbackCapture.capture(), + ArgumentMatchers.any() )).thenAnswer(_ => { callbackCapture.getValue.apply(( error, @@ -355,7 +356,8 @@ class CoordinatorPartitionWriterTest { tp, "transactional-id", 10L, - 5.toShort + 5.toShort, + ApiKeys.TXN_OFFSET_COMMIT.latestVersion() ) if (error == Errors.NONE) { diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorAdapterTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorAdapterTest.scala index 14b41b09f8..647d6072fc 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorAdapterTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorAdapterTest.scala @@ -761,7 +761,8 @@ class GroupCoordinatorAdapterTest { ) )), capturedCallback.capture(), - ArgumentMatchers.eq(RequestLocal(bufferSupplier)) + ArgumentMatchers.eq(RequestLocal(bufferSupplier)), + ArgumentMatchers.any() ) capturedCallback.getValue.apply(Map( diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala index be0038c027..9bed515aed 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala @@ -24,13 +24,13 @@ import kafka.common.OffsetAndMetadata import kafka.coordinator.AbstractCoordinatorConcurrencyTest import kafka.coordinator.AbstractCoordinatorConcurrencyTest._ import kafka.coordinator.group.GroupCoordinatorConcurrencyTest._ -import kafka.server.{DelayedOperationPurgatory, KafkaConfig, KafkaRequestHandler} +import kafka.server.{DelayedOperationPurgatory, KafkaConfig, KafkaRequestHandler, RequestLocal} import kafka.utils.CoreUtils import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity -import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetFetchResponse} import org.apache.kafka.common.utils.Time import org.junit.jupiter.api.Assertions._ @@ -319,7 +319,7 @@ class GroupCoordinatorConcurrencyTest extends AbstractCoordinatorConcurrencyTest // Since the replica manager is mocked we can use a dummy value for transactionalId. groupCoordinator.handleTxnCommitOffsets(member.group.groupId, "dummy-txn-id", producerId, producerEpoch, JoinGroupRequest.UNKNOWN_MEMBER_ID, Option.empty, JoinGroupRequest.UNKNOWN_GENERATION_ID, - offsets, callbackWithTxnCompletion) + offsets, callbackWithTxnCompletion, RequestLocal.NoCaching, ApiKeys.TXN_OFFSET_COMMIT.latestVersion()) replicaManager.tryCompleteActions() } finally lock.foreach(_.unlock()) } 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 7930a798c0..0f8190b17c 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -22,7 +22,7 @@ import kafka.common.OffsetAndMetadata import kafka.server.{ActionQueue, DelayedOperationPurgatory, HostedPartition, KafkaConfig, KafkaRequestHandler, ReplicaManager, RequestLocal} import kafka.utils._ import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} -import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{MemoryRecords, RecordBatch} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest, OffsetFetchResponse, TransactionResult} @@ -3844,6 +3844,7 @@ class GroupCoordinatorTest { verifyErrors(Errors.INVALID_PRODUCER_ID_MAPPING, Errors.INVALID_PRODUCER_ID_MAPPING) verifyErrors(Errors.INVALID_TXN_STATE, Errors.INVALID_TXN_STATE) + verifyErrors(Errors.NETWORK_EXCEPTION, Errors.COORDINATOR_LOAD_IN_PROGRESS) verifyErrors(Errors.NOT_ENOUGH_REPLICAS, Errors.COORDINATOR_NOT_AVAILABLE) verifyErrors(Errors.NOT_LEADER_OR_FOLLOWER, Errors.NOT_COORDINATOR) verifyErrors(Errors.KAFKA_STORAGE_ERROR, Errors.NOT_COORDINATOR) @@ -4173,7 +4174,8 @@ class GroupCoordinatorTest { ArgumentMatchers.eq(producerId), ArgumentMatchers.eq(producerEpoch), any(), - postVerificationCallback.capture() + postVerificationCallback.capture(), + any() )).thenAnswer( _ => postVerificationCallback.getValue()((verificationError, VerificationGuard.SENTINEL)) ) @@ -4198,7 +4200,7 @@ class GroupCoordinatorTest { when(replicaManager.getMagic(any[TopicPartition])).thenReturn(Some(RecordBatch.MAGIC_VALUE_V2)) groupCoordinator.handleTxnCommitOffsets(groupId, transactionalId, producerId, producerEpoch, - memberId, groupInstanceId, generationId, offsets, responseCallback) + memberId, groupInstanceId, generationId, offsets, responseCallback, RequestLocal.NoCaching, ApiKeys.TXN_OFFSET_COMMIT.latestVersion()) val result = Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS)) result } diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala index 1efccc847c..1da8d66480 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala @@ -283,7 +283,7 @@ class TransactionCoordinatorTest { coordinator.handleVerifyPartitionsInTransaction(transactionalId, 0L, 0, partitions, verifyPartitionsInTxnCallback) errors.foreach { case (_, error) => - assertEquals(Errors.INVALID_TXN_STATE, error) + assertEquals(Errors.ABORTABLE_TRANSACTION, error) } } @@ -399,7 +399,7 @@ class TransactionCoordinatorTest { val extraPartitions = partitions ++ Set(new TopicPartition("topic2", 0)) coordinator.handleVerifyPartitionsInTransaction(transactionalId, 0L, 0, extraPartitions, verifyPartitionsInTxnCallback) - assertEquals(Errors.INVALID_TXN_STATE, errors(new TopicPartition("topic2", 0))) + assertEquals(Errors.ABORTABLE_TRANSACTION, errors(new TopicPartition("topic2", 0))) assertEquals(Errors.NONE, errors(new TopicPartition("topic1", 0))) verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId)) } diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala index de58f8ed7f..3356c4f9e3 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala @@ -34,7 +34,7 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.mockito.ArgumentMatchers.any import org.mockito.{ArgumentCaptor, ArgumentMatchers} -import org.mockito.Mockito.{mock, mockConstruction, times, verify, verifyNoMoreInteractions, when} +import org.mockito.Mockito.{clearInvocations, mock, mockConstruction, times, verify, verifyNoMoreInteractions, when} import scala.jdk.CollectionConverters._ import scala.collection.mutable @@ -59,6 +59,7 @@ class TransactionMarkerChannelManagerTest { private val txnTopicPartition1 = 0 private val txnTopicPartition2 = 1 private val coordinatorEpoch = 0 + private val coordinatorEpoch2 = 1 private val txnTimeoutMs = 0 private val txnResult = TransactionResult.COMMIT private val txnMetadata1 = new TransactionMetadata(transactionalId1, producerId1, producerId1, producerEpoch, lastProducerEpoch, @@ -177,6 +178,86 @@ class TransactionMarkerChannelManagerTest { any()) } + @Test + def shouldNotLoseTxnCompletionAfterLoad(): Unit = { + mockCache() + + val expectedTransition = txnMetadata2.prepareComplete(time.milliseconds()) + + when(metadataCache.getPartitionLeaderEndpoint( + ArgumentMatchers.eq(partition1.topic), + ArgumentMatchers.eq(partition1.partition), + any()) + ).thenReturn(Some(broker1)) + + // Build a successful client response. + val header = new RequestHeader(ApiKeys.WRITE_TXN_MARKERS, 0, "client", 1) + val successfulResponse = new WriteTxnMarkersResponse( + Collections.singletonMap(producerId2: java.lang.Long, Collections.singletonMap(partition1, Errors.NONE))) + val successfulClientResponse = new ClientResponse(header, null, null, + time.milliseconds(), time.milliseconds(), false, null, null, + successfulResponse) + + // Build a disconnected client response. + val disconnectedClientResponse = new ClientResponse(header, null, null, + time.milliseconds(), time.milliseconds(), true, null, null, + null) + + // Test matrix to cover various scenarios: + val clientResponses = Seq(successfulClientResponse, disconnectedClientResponse) + val getTransactionStateResponses = Seq( + // NOT_COORDINATOR error case + Left(Errors.NOT_COORDINATOR), + // COORDINATOR_LOAD_IN_PROGRESS + Left(Errors.COORDINATOR_LOAD_IN_PROGRESS), + // "Newly loaded" transaction state with the new epoch. + Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch2, txnMetadata2))) + ) + + clientResponses.foreach { clientResponse => + getTransactionStateResponses.foreach { getTransactionStateResponse => + // Reset data from previous iteration. + txnMetadata2.topicPartitions.add(partition1) + clearInvocations(txnStateManager) + // Send out markers for a transaction before load. + channelManager.addTxnMarkersToSend(coordinatorEpoch, txnResult, + txnMetadata2, expectedTransition) + + // Drain the marker to make it "in-flight". + val requests1 = channelManager.generateRequests().asScala + assertEquals(1, requests1.size) + + // Simulate a partition load: + // 1. Remove the markers from the channel manager. + // 2. Simulate the corresponding test case scenario. + // 3. Add the markers back to the channel manager. + channelManager.removeMarkersForTxnTopicPartition(txnTopicPartition2) + when(txnStateManager.getTransactionState(ArgumentMatchers.eq(transactionalId2))) + .thenReturn(getTransactionStateResponse) + channelManager.addTxnMarkersToSend(coordinatorEpoch2, txnResult, + txnMetadata2, expectedTransition) + + // Complete the marker from the previous epoch. + requests1.head.handler.onComplete(clientResponse) + + // Now drain and complete the marker from the new epoch. + when(txnStateManager.getTransactionState(ArgumentMatchers.eq(transactionalId2))) + .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch2, txnMetadata2)))) + val requests2 = channelManager.generateRequests().asScala + assertEquals(1, requests2.size) + requests2.head.handler.onComplete(successfulClientResponse) + + verify(txnStateManager).appendTransactionToLog( + ArgumentMatchers.eq(transactionalId2), + ArgumentMatchers.eq(coordinatorEpoch2), + ArgumentMatchers.eq(expectedTransition), + capturedErrorsCallback.capture(), + any(), + any()) + } + } + } + @Test def shouldGenerateEmptyMapWhenNoRequestsOutstanding(): Unit = { assertTrue(channelManager.generateRequests().isEmpty) diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala index aecf6542f7..1004915f46 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala @@ -18,7 +18,6 @@ package kafka.coordinator.transaction import java.{lang, util} import java.util.Arrays.asList - import org.apache.kafka.clients.ClientResponse import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -43,18 +42,19 @@ class TransactionMarkerRequestCompletionHandlerTest { private val coordinatorEpoch = 0 private val txnResult = TransactionResult.COMMIT private val topicPartition = new TopicPartition("topic1", 0) - private val txnIdAndMarkers = asList( - TxnIdAndMarkerEntry(transactionalId, new WriteTxnMarkersRequest.TxnMarkerEntry(producerId, producerEpoch, coordinatorEpoch, txnResult, asList(topicPartition)))) - private val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, producerEpoch, lastProducerEpoch, txnTimeoutMs, PrepareCommit, mutable.Set[TopicPartition](topicPartition), 0L, 0L) + private val pendingCompleteTxnAndMarkers = asList( + PendingCompleteTxnAndMarkerEntry( + PendingCompleteTxn(transactionalId, coordinatorEpoch, txnMetadata, txnMetadata.prepareComplete(42)), + new WriteTxnMarkersRequest.TxnMarkerEntry(producerId, producerEpoch, coordinatorEpoch, txnResult, asList(topicPartition)))) private val markerChannelManager: TransactionMarkerChannelManager = mock(classOf[TransactionMarkerChannelManager]) private val txnStateManager: TransactionStateManager = mock(classOf[TransactionStateManager]) - private val handler = new TransactionMarkerRequestCompletionHandler(brokerId, txnStateManager, markerChannelManager, txnIdAndMarkers) + private val handler = new TransactionMarkerRequestCompletionHandler(brokerId, txnStateManager, markerChannelManager, pendingCompleteTxnAndMarkers) private def mockCache(): Unit = { when(txnStateManager.partitionFor(transactionalId)) @@ -70,8 +70,9 @@ class TransactionMarkerRequestCompletionHandlerTest { handler.onComplete(new ClientResponse(new RequestHeader(ApiKeys.PRODUCE, 0, "client", 1), null, null, 0, 0, true, null, null, null)) - verify(markerChannelManager).addTxnMarkersToBrokerQueue(transactionalId, - producerId, producerEpoch, txnResult, coordinatorEpoch, Set[TopicPartition](topicPartition)) + verify(markerChannelManager).addTxnMarkersToBrokerQueue(producerId, + producerEpoch, txnResult, pendingCompleteTxnAndMarkers.get(0).pendingCompleteTxn, + Set[TopicPartition](topicPartition)) } @Test @@ -193,8 +194,9 @@ class TransactionMarkerRequestCompletionHandlerTest { null, null, 0, 0, false, null, null, response)) assertEquals(txnMetadata.topicPartitions, mutable.Set[TopicPartition](topicPartition)) - verify(markerChannelManager).addTxnMarkersToBrokerQueue(transactionalId, - producerId, producerEpoch, txnResult, coordinatorEpoch, Set[TopicPartition](topicPartition)) + verify(markerChannelManager).addTxnMarkersToBrokerQueue(producerId, + producerEpoch, txnResult, pendingCompleteTxnAndMarkers.get(0).pendingCompleteTxn, + Set[TopicPartition](topicPartition)) } private def verifyThrowIllegalStateExceptionOnError(error: Errors) = { @@ -222,7 +224,8 @@ class TransactionMarkerRequestCompletionHandlerTest { private def verifyRemoveDelayedOperationOnError(error: Errors): Unit = { var removed = false - when(markerChannelManager.removeMarkersForTxnId(transactionalId)) + val pendingCompleteTxn = pendingCompleteTxnAndMarkers.get(0).pendingCompleteTxn + when(markerChannelManager.removeMarkersForTxn(pendingCompleteTxn)) .thenAnswer(_ => removed = true) val response = new WriteTxnMarkersResponse(createProducerIdErrorMap(error)) diff --git a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala index 29b5fd34f9..bffd41156b 100644 --- a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala @@ -100,7 +100,7 @@ class LocalLogTest { initialOffset: Long = 0L): Unit = { log.append(lastOffset = initialOffset + records.size - 1, largestTimestamp = records.head.timestamp, - shallowOffsetOfMaxTimestamp = initialOffset, + offsetOfMaxTimestamp = initialOffset, records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*)) } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala index 8729045db7..49e518ac2a 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala @@ -251,13 +251,13 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati def kafkaConfigWithCleanerConfig(cleanerConfig: CleanerConfig): KafkaConfig = { val props = TestUtils.createBrokerConfig(0, "localhost:2181") - props.put(KafkaConfig.LogCleanerThreadsProp, cleanerConfig.numThreads.toString) - props.put(KafkaConfig.LogCleanerDedupeBufferSizeProp, cleanerConfig.dedupeBufferSize.toString) - props.put(KafkaConfig.LogCleanerDedupeBufferLoadFactorProp, cleanerConfig.dedupeBufferLoadFactor.toString) - props.put(KafkaConfig.LogCleanerIoBufferSizeProp, cleanerConfig.ioBufferSize.toString) + props.put(CleanerConfig.LOG_CLEANER_THREADS_PROP, cleanerConfig.numThreads.toString) + props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, cleanerConfig.dedupeBufferSize.toString) + props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, cleanerConfig.dedupeBufferLoadFactor.toString) + props.put(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, cleanerConfig.ioBufferSize.toString) props.put(KafkaConfig.MessageMaxBytesProp, cleanerConfig.maxMessageSize.toString) - props.put(KafkaConfig.LogCleanerBackoffMsProp, cleanerConfig.backoffMs.toString) - props.put(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp, cleanerConfig.maxIoBytesPerSecond.toString) + props.put(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, cleanerConfig.backoffMs.toString) + props.put(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, cleanerConfig.maxIoBytesPerSecond.toString) KafkaConfig.fromProps(props) } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 9ad6f99fb6..df2cb8f7c5 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -1996,7 +1996,7 @@ class LogCleanerTest extends Logging { @Test def testReconfigureLogCleanerIoMaxBytesPerSecond(): Unit = { val oldKafkaProps = TestUtils.createBrokerConfig(1, "localhost:2181") - oldKafkaProps.setProperty(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp, "10000000") + oldKafkaProps.setProperty(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, "10000000") val logCleaner = new LogCleaner(LogCleaner.cleanerConfig(new KafkaConfig(oldKafkaProps)), logDirs = Array(TestUtils.tempDir()), @@ -2010,14 +2010,14 @@ class LogCleanerTest extends Logging { } try { - assertEquals(10000000, logCleaner.throttler.desiredRatePerSec, s"Throttler.desiredRatePerSec should be initialized from initial `${KafkaConfig.LogCleanerIoMaxBytesPerSecondProp}` config.") + assertEquals(10000000, logCleaner.throttler.desiredRatePerSec, s"Throttler.desiredRatePerSec should be initialized from initial `${CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP}` config.") val newKafkaProps = TestUtils.createBrokerConfig(1, "localhost:2181") - newKafkaProps.setProperty(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp, "20000000") + newKafkaProps.setProperty(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, "20000000") logCleaner.reconfigure(new KafkaConfig(oldKafkaProps), new KafkaConfig(newKafkaProps)) - assertEquals(20000000, logCleaner.throttler.desiredRatePerSec, s"Throttler.desiredRatePerSec should be updated with new `${KafkaConfig.LogCleanerIoMaxBytesPerSecondProp}` config.") + assertEquals(20000000, logCleaner.throttler.desiredRatePerSec, s"Throttler.desiredRatePerSec should be updated with new `${CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP}` config.") } finally { logCleaner.shutdown() } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 5dcca54961..c32e59e715 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -34,14 +34,14 @@ import org.apache.kafka.server.common.MetadataVersion.IBP_0_11_0_IV0 import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.util.{MockTime, Scheduler} import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache -import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, EpochEntry, FetchDataInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile} +import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, EpochEntry, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile} import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue} import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.CsvSource -import org.mockito.ArgumentMatchers +import org.mockito.{ArgumentMatchers, Mockito} import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{mock, reset, times, verify, when} @@ -352,19 +352,15 @@ class LogLoaderTest { // Intercept all segment read calls val interceptedLogSegments = new LogSegments(topicPartition) { override def add(segment: LogSegment): LogSegment = { - val wrapper = new LogSegment(segment) { - - override def read(startOffset: Long, maxSize: Int, maxPosition: Long, minOneMessage: Boolean): FetchDataInfo = { - segmentsWithReads += this - super.read(startOffset, maxSize, maxPosition, minOneMessage) - } - - override def recover(producerStateManager: ProducerStateManager, - leaderEpochCache: Optional[LeaderEpochFileCache]): Int = { - recoveredSegments += this - super.recover(producerStateManager, leaderEpochCache) - } - } + val wrapper = Mockito.spy(segment) + Mockito.doAnswer { in => + segmentsWithReads += wrapper + segment.read(in.getArgument(0, classOf[java.lang.Long]), in.getArgument(1, classOf[java.lang.Integer]), in.getArgument(2, classOf[java.lang.Long]), in.getArgument(3, classOf[java.lang.Boolean])) + }.when(wrapper).read(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any()) + Mockito.doAnswer { in => + recoveredSegments += wrapper + segment.recover(in.getArgument(0, classOf[ProducerStateManager]), in.getArgument(1, classOf[Optional[LeaderEpochFileCache]])) + }.when(wrapper).recover(ArgumentMatchers.any(), ArgumentMatchers.any()) super.add(wrapper) } } diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 2dcc3e5a35..ce6e2a3b04 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -46,7 +46,7 @@ import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, Future} import java.util.{Collections, Optional, OptionalLong, Properties} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.util.MockTime -import org.apache.kafka.storage.internals.log.{FetchDataInfo, FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig, RemoteIndexCache} +import org.apache.kafka.storage.internals.log.{FetchDataInfo, FetchIsolation, LogConfig, LogDirFailureChannel, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache} import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler import scala.collection.{Map, mutable} @@ -1117,6 +1117,84 @@ class LogManagerTest { assertEquals(2, logManager.directoryIdsSet.size) } + @Test + def testCheckpointLogStartOffsetForRemoteTopic(): Unit = { + logManager.shutdown() + + val props = new Properties() + props.putAll(logProps) + props.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + val logConfig = new LogConfig(props) + logManager = TestUtils.createLogManager( + defaultConfig = logConfig, + configRepository = new MockConfigRepository, + logDirs = Seq(this.logDir), + time = this.time, + recoveryThreadsPerDataDir = 1, + remoteStorageSystemEnable = true + ) + + val checkpointFile = new File(logDir, LogManager.LogStartOffsetCheckpointFile) + val checkpoint = new OffsetCheckpointFile(checkpointFile) + val topicPartition = new TopicPartition("test", 0) + val log = logManager.getOrCreateLog(topicPartition, topicId = None) + var offset = 0L + for(_ <- 0 until 50) { + val set = TestUtils.singletonRecords("test".getBytes()) + val info = log.appendAsLeader(set, leaderEpoch = 0) + offset = info.lastOffset + if (offset != 0 && offset % 10 == 0) + log.roll() + } + assertEquals(5, log.logSegments.size()) + log.updateHighWatermark(49) + // simulate calls to upload 3 segments to remote storage and remove them from local-log. + log.updateHighestOffsetInRemoteStorage(30) + log.maybeIncrementLocalLogStartOffset(31L, LogStartOffsetIncrementReason.SegmentDeletion) + log.deleteOldSegments() + assertEquals(2, log.logSegments.size()) + + // simulate two remote-log segment deletion + val logStartOffset = 21L + log.maybeIncrementLogStartOffset(logStartOffset, LogStartOffsetIncrementReason.SegmentDeletion) + logManager.checkpointLogStartOffsets() + + assertEquals(logStartOffset, log.logStartOffset) + assertEquals(logStartOffset, checkpoint.read().getOrElse(topicPartition, -1L)) + } + + @Test + def testCheckpointLogStartOffsetForNormalTopic(): Unit = { + val checkpointFile = new File(logDir, LogManager.LogStartOffsetCheckpointFile) + val checkpoint = new OffsetCheckpointFile(checkpointFile) + val topicPartition = new TopicPartition("test", 0) + val log = logManager.getOrCreateLog(topicPartition, topicId = None) + var offset = 0L + for(_ <- 0 until 50) { + val set = TestUtils.singletonRecords("test".getBytes()) + val info = log.appendAsLeader(set, leaderEpoch = 0) + offset = info.lastOffset + if (offset != 0 && offset % 10 == 0) + log.roll() + } + assertEquals(5, log.logSegments.size()) + log.updateHighWatermark(49) + + val logStartOffset = 31L + log.maybeIncrementLogStartOffset(logStartOffset, LogStartOffsetIncrementReason.SegmentDeletion) + logManager.checkpointLogStartOffsets() + assertEquals(5, log.logSegments.size()) + assertEquals(logStartOffset, checkpoint.read().getOrElse(topicPartition, -1L)) + + log.deleteOldSegments() + assertEquals(2, log.logSegments.size()) + assertEquals(logStartOffset, log.logStartOffset) + + // When you checkpoint log-start-offset after removing the segments, then there should not be any checkpoint + logManager.checkpointLogStartOffsets() + assertEquals(-1L, checkpoint.read().getOrElse(topicPartition, -1L)) + } + def writeMetaProperties(dir: File, directoryId: Optional[Uuid] = Optional.empty()): Unit = { val metaProps = new MetaProperties.Builder(). setVersion(MetaPropertiesVersion.V0). diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index b4e278c380..bb0c85a858 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -86,10 +86,10 @@ class LogSegmentTest { def testAppendForLogSegmentOffsetOverflowException(baseOffset: Long, largestOffset: Long): Unit = { val seg = createSegment(baseOffset) val currentTime = Time.SYSTEM.milliseconds() - val shallowOffsetOfMaxTimestamp = largestOffset + val offsetOfMaxTimestamp = largestOffset val memoryRecords = records(0, "hello") assertThrows(classOf[LogSegmentOffsetOverflowException], () => { - seg.append(largestOffset, currentTime, shallowOffsetOfMaxTimestamp, memoryRecords) + seg.append(largestOffset, currentTime, offsetOfMaxTimestamp, memoryRecords) }) } diff --git a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala index 0b5d17cbcc..53b385c62e 100644 --- a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala @@ -173,9 +173,9 @@ class LogValidatorTest { assertEquals(now, validatedResults.maxTimestampMs, s"Max timestamp should be $now") assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed") - // we index from last offset in version 2 instead of base offset - val expectedMaxTimestampOffset = if (magic >= RecordBatch.MAGIC_VALUE_V2) 2 else 0 - assertEquals(expectedMaxTimestampOffset, validatedResults.shallowOffsetOfMaxTimestampMs, + // If it's LOG_APPEND_TIME, the offset will be the offset of the first record + val expectedMaxTimestampOffset = 0 + assertEquals(expectedMaxTimestampOffset, validatedResults.offsetOfMaxTimestampMs, s"The offset of max timestamp should be $expectedMaxTimestampOffset") verifyRecordValidationStats(validatedResults.recordValidationStats, numConvertedRecords = 0, records, compressed = false) @@ -219,8 +219,8 @@ class LogValidatorTest { "MessageSet should still valid") assertEquals(now, validatedResults.maxTimestampMs, s"Max timestamp should be $now") - assertEquals(records.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, - s"The offset of max timestamp should be ${records.records.asScala.size - 1}") + assertEquals(0, validatedResults.offsetOfMaxTimestampMs, + s"The offset of max timestamp should be 0 if logAppendTime is used") assertTrue(validatedResults.messageSizeMaybeChanged, "Message size may have been changed") @@ -271,8 +271,8 @@ class LogValidatorTest { "MessageSet should still valid") assertEquals(now, validatedResults.maxTimestampMs, s"Max timestamp should be $now") - assertEquals(records.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, - s"The offset of max timestamp should be ${records.records.asScala.size - 1}") + assertEquals(0, validatedResults.offsetOfMaxTimestampMs, + s"The offset of max timestamp should be 0 if logAppendTime is used") assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed") @@ -341,6 +341,7 @@ class LogValidatorTest { private def checkNonCompressed(magic: Byte): Unit = { val now = System.currentTimeMillis() + // set the timestamp of seq(1) (i.e. offset 1) as the max timestamp val timestampSeq = Seq(now - 1, now + 1, now) val (producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) = @@ -403,14 +404,8 @@ class LogValidatorTest { assertEquals(now + 1, validatingResults.maxTimestampMs, s"Max timestamp should be ${now + 1}") - val expectedShallowOffsetOfMaxTimestamp = if (magic >= RecordVersion.V2.value) { - // v2 records are always batched, even when not compressed. - // the shallow offset of max timestamp is the last offset of the batch - recordList.size - 1 - } else { - 1 - } - assertEquals(expectedShallowOffsetOfMaxTimestamp, validatingResults.shallowOffsetOfMaxTimestampMs, + val expectedOffsetOfMaxTimestamp = 1 + assertEquals(expectedOffsetOfMaxTimestamp, validatingResults.offsetOfMaxTimestampMs, s"Offset of max timestamp should be 1") assertFalse(validatingResults.messageSizeMaybeChanged, @@ -431,6 +426,7 @@ class LogValidatorTest { private def checkRecompression(magic: Byte): Unit = { val now = System.currentTimeMillis() + // set the timestamp of seq(1) (i.e. offset 1) as the max timestamp val timestampSeq = Seq(now - 1, now + 1, now) val (producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) = @@ -484,8 +480,8 @@ class LogValidatorTest { } assertEquals(now + 1, validatingResults.maxTimestampMs, s"Max timestamp should be ${now + 1}") - assertEquals(2, validatingResults.shallowOffsetOfMaxTimestampMs, - "Offset of max timestamp should be 2") + assertEquals(1, validatingResults.offsetOfMaxTimestampMs, + "Offset of max timestamp should be 1") assertTrue(validatingResults.messageSizeMaybeChanged, "Message size should have been changed") @@ -536,8 +532,8 @@ class LogValidatorTest { } assertEquals(validatedResults.maxTimestampMs, RecordBatch.NO_TIMESTAMP, s"Max timestamp should be ${RecordBatch.NO_TIMESTAMP}") - assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, - s"Offset of max timestamp should be ${validatedRecords.records.asScala.size - 1}") + assertEquals(-1, validatedResults.offsetOfMaxTimestampMs, + s"Offset of max timestamp should be -1") assertTrue(validatedResults.messageSizeMaybeChanged, "Message size should have been changed") verifyRecordValidationStats(validatedResults.recordValidationStats, numConvertedRecords = 3, records, @@ -583,8 +579,8 @@ class LogValidatorTest { assertEquals(RecordBatch.NO_SEQUENCE, batch.baseSequence) } assertEquals(timestamp, validatedResults.maxTimestampMs) - assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, - s"Offset of max timestamp should be ${validatedRecords.records.asScala.size - 1}") + assertEquals(0, validatedResults.offsetOfMaxTimestampMs, + s"Offset of max timestamp should be 0 when multiple records having the same max timestamp.") assertTrue(validatedResults.messageSizeMaybeChanged, "Message size should have been changed") verifyRecordValidationStats(validatedResults.recordValidationStats, numConvertedRecords = 3, records, @@ -598,6 +594,7 @@ class LogValidatorTest { private def checkCompressed(magic: Byte): Unit = { val now = System.currentTimeMillis() + // set the timestamp of seq(1) (i.e. offset 1) as the max timestamp val timestampSeq = Seq(now - 1, now + 1, now) val (producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) = @@ -654,11 +651,9 @@ class LogValidatorTest { } assertEquals(now + 1, validatedResults.maxTimestampMs, s"Max timestamp should be ${now + 1}") - // All versions have an outer batch when compressed, so the shallow offset - // of max timestamp is always the offset of the last record in the batch. - val expectedShallowOffsetOfMaxTimestamp = recordList.size - 1 - assertEquals(expectedShallowOffsetOfMaxTimestamp, validatedResults.shallowOffsetOfMaxTimestampMs, - s"Offset of max timestamp should be ${validatedRecords.records.asScala.size - 1}") + val expectedOffsetOfMaxTimestamp = 1 + assertEquals(expectedOffsetOfMaxTimestamp, validatedResults.offsetOfMaxTimestampMs, + s"Offset of max timestamp should be 1") assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed") verifyRecordValidationStats(validatedResults.recordValidationStats, numConvertedRecords = 0, records, diff --git a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala index 03aa847ded..810e0b1e4e 100644 --- a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala @@ -579,6 +579,12 @@ class ProducerStateManagerTest { assertTrue(logDir.list().head.nonEmpty, "Snapshot file is empty") } + @Test + def testFetchSnapshotEmptySnapShot(): Unit = { + val offset = 1 + assertEquals(Optional.empty(), stateManager.fetchSnapshot(offset)) + } + @Test def testRecoverFromSnapshotUnfinishedTransaction(): Unit = { val epoch = 0.toShort diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 9ee2c61e23..4cc06b083f 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -2127,6 +2127,92 @@ class UnifiedLogTest { log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(remoteLogManager))) } + @Test + def testFetchLatestTieredTimestampNoRemoteStorage(): Unit = { + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) + val log = createLog(logDir, logConfig) + + assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, -1, Optional.of(-1))), + log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIERED_TIMESTAMP)) + + val firstTimestamp = mockTime.milliseconds + val leaderEpoch = 0 + log.appendAsLeader(TestUtils.singletonRecords( + value = TestUtils.randomBytes(10), + timestamp = firstTimestamp), + leaderEpoch = leaderEpoch) + + val secondTimestamp = firstTimestamp + 1 + log.appendAsLeader(TestUtils.singletonRecords( + value = TestUtils.randomBytes(10), + timestamp = secondTimestamp), + leaderEpoch = leaderEpoch) + + assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, -1, Optional.of(-1))), + log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIERED_TIMESTAMP)) + } + + @Test + def testFetchLatestTieredTimestampWithRemoteStorage(): Unit = { + val remoteLogManager = mock(classOf[RemoteLogManager]) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1, + remoteLogStorageEnable = true) + val log = createLog(logDir, logConfig, remoteStorageSystemEnable = true, remoteLogManager = Some(remoteLogManager)) + when(remoteLogManager.findOffsetByTimestamp(log.topicPartition, 0, 0, log.leaderEpochCache.get)) + .thenReturn(Optional.empty[TimestampAndOffset]()) + + assertEquals(None, log.fetchOffsetByTimestamp(0L, Some(remoteLogManager))) + assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0, Optional.empty())), + log.fetchOffsetByTimestamp(ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, Some(remoteLogManager))) + + val firstTimestamp = mockTime.milliseconds + val firstLeaderEpoch = 0 + log.appendAsLeader(TestUtils.singletonRecords( + value = TestUtils.randomBytes(10), + timestamp = firstTimestamp), + leaderEpoch = firstLeaderEpoch) + + val secondTimestamp = firstTimestamp + 1 + val secondLeaderEpoch = 1 + log.appendAsLeader(TestUtils.singletonRecords( + value = TestUtils.randomBytes(10), + timestamp = secondTimestamp), + leaderEpoch = secondLeaderEpoch) + + when(remoteLogManager.findOffsetByTimestamp(ArgumentMatchers.eq(log.topicPartition), + anyLong(), anyLong(), ArgumentMatchers.eq(log.leaderEpochCache.get))) + .thenAnswer(ans => { + val timestamp = ans.getArgument(1).asInstanceOf[Long] + Optional.of(timestamp) + .filter(_ == firstTimestamp) + .map[TimestampAndOffset](x => new TimestampAndOffset(x, 0L, Optional.of(firstLeaderEpoch))) + }) + log._localLogStartOffset = 1 + log._highestOffsetInRemoteStorage = 0 + + // In the assertions below we test that offset 0 (first timestamp) is in remote and offset 1 (second timestamp) is in local storage. + assertEquals(Some(new TimestampAndOffset(firstTimestamp, 0L, Optional.of(firstLeaderEpoch))), + log.fetchOffsetByTimestamp(firstTimestamp, Some(remoteLogManager))) + assertEquals(Some(new TimestampAndOffset(secondTimestamp, 1L, Optional.of(secondLeaderEpoch))), + log.fetchOffsetByTimestamp(secondTimestamp, Some(remoteLogManager))) + + assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0L, Optional.of(firstLeaderEpoch))), + log.fetchOffsetByTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP, Some(remoteLogManager))) + assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 0L, Optional.of(firstLeaderEpoch))), + log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIERED_TIMESTAMP, Some(remoteLogManager))) + assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 1L, Optional.of(secondLeaderEpoch))), + log.fetchOffsetByTimestamp(ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, Some(remoteLogManager))) + assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 2L, Optional.of(secondLeaderEpoch))), + log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(remoteLogManager))) + + // The cache can be updated directly after a leader change. + // The new latest offset should reflect the updated epoch. + log.maybeAssignEpochStartOffset(2, 2L) + + assertEquals(Some(new TimestampAndOffset(ListOffsetsResponse.UNKNOWN_TIMESTAMP, 2L, Optional.of(2))), + log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP, Some(remoteLogManager))) + } + /** * Test the Log truncate operations */ diff --git a/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala b/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala index 48b01503bb..ff65868674 100644 --- a/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala +++ b/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala @@ -340,6 +340,8 @@ class RemoteIndexCacheTest { val spyEntry = generateSpyCacheEntry() cache.internalCache.put(rlsMetadata.remoteLogSegmentId().id(), spyEntry) + TestUtils.waitUntilTrue(() => cache.cleanerThread().isStarted, "Cleaner thread should be started") + // close the cache cache.close() diff --git a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnManagerTest.scala b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnManagerTest.scala index 9e34322ec9..5ee9cf8302 100644 --- a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnManagerTest.scala @@ -24,11 +24,12 @@ import org.apache.kafka.clients.{ClientResponse, NetworkClient} import org.apache.kafka.common.errors.{AuthenticationException, SaslAuthenticationException, UnsupportedVersionException} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.{AddPartitionsToTxnTopic, AddPartitionsToTxnTopicCollection, AddPartitionsToTxnTransaction, AddPartitionsToTxnTransactionCollection} -import org.apache.kafka.common.message.{AddPartitionsToTxnResponseData, MetadataResponseData} +import org.apache.kafka.common.message.AddPartitionsToTxnResponseData import org.apache.kafka.common.message.AddPartitionsToTxnResponseData.AddPartitionsToTxnResultCollection +import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState import org.apache.kafka.common.{Node, TopicPartition} import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.{AbstractResponse, AddPartitionsToTxnRequest, AddPartitionsToTxnResponse} +import org.apache.kafka.common.requests.{AbstractResponse, AddPartitionsToTxnRequest, AddPartitionsToTxnResponse, MetadataResponse} import org.apache.kafka.common.utils.MockTime import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.RequestAndCompletionHandler @@ -70,6 +71,7 @@ class AddPartitionsToTxnManagerTest { private val authenticationErrorResponse = clientResponse(null, authException = new SaslAuthenticationException("")) private val versionMismatchResponse = clientResponse(null, mismatchException = new UnsupportedVersionException("")) private val disconnectedResponse = clientResponse(null, disconnected = true) + private val supportedOperation = genericError private val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:2181")) @@ -98,31 +100,16 @@ class AddPartitionsToTxnManagerTest { when(partitionFor.apply(transactionalId1)).thenReturn(0) when(partitionFor.apply(transactionalId2)).thenReturn(1) when(partitionFor.apply(transactionalId3)).thenReturn(0) - when(metadataCache.getTopicMetadata(Set(Topic.TRANSACTION_STATE_TOPIC_NAME), config.interBrokerListenerName)) - .thenReturn(Seq( - new MetadataResponseData.MetadataResponseTopic() - .setName(Topic.TRANSACTION_STATE_TOPIC_NAME) - .setPartitions(List( - new MetadataResponseData.MetadataResponsePartition() - .setPartitionIndex(0) - .setLeaderId(0), - new MetadataResponseData.MetadataResponsePartition() - .setPartitionIndex(1) - .setLeaderId(1) - ).asJava) - )) - when(metadataCache.getAliveBrokerNode(0, config.interBrokerListenerName)) - .thenReturn(Some(node0)) - when(metadataCache.getAliveBrokerNode(1, config.interBrokerListenerName)) - .thenReturn(Some(node1)) + mockTransactionStateMetadata(0, 0, Some(node0)) + mockTransactionStateMetadata(1, 1, Some(node1)) val transaction1Errors = mutable.Map[TopicPartition, Errors]() val transaction2Errors = mutable.Map[TopicPartition, Errors]() val transaction3Errors = mutable.Map[TopicPartition, Errors]() - addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1Errors)) - addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transaction2Errors)) - addPartitionsToTxnManager.verifyTransaction(transactionalId3, producerId3, producerEpoch = 0, topicPartitions, setErrors(transaction3Errors)) + addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1Errors), supportedOperation) + addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transaction2Errors), supportedOperation) + addPartitionsToTxnManager.verifyTransaction(transactionalId3, producerId3, producerEpoch = 0, topicPartitions, setErrors(transaction3Errors), supportedOperation) // We will try to add transaction1 3 more times (retries). One will have the same epoch, one will have a newer epoch, and one will have an older epoch than the new one we just added. val transaction1RetryWithSameEpochErrors = mutable.Map[TopicPartition, Errors]() @@ -130,17 +117,17 @@ class AddPartitionsToTxnManagerTest { val transaction1RetryWithOldEpochErrors = mutable.Map[TopicPartition, Errors]() // Trying to add more transactional data for the same transactional ID, producer ID, and epoch should simply replace the old data and send a retriable response. - addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1RetryWithSameEpochErrors)) + addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1RetryWithSameEpochErrors), supportedOperation) val expectedNetworkErrors = topicPartitions.map(_ -> Errors.NETWORK_EXCEPTION).toMap assertEquals(expectedNetworkErrors, transaction1Errors) // Trying to add more transactional data for the same transactional ID and producer ID, but new epoch should replace the old data and send an error response for it. - addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 1, topicPartitions, setErrors(transaction1RetryWithNewerEpochErrors)) + addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 1, topicPartitions, setErrors(transaction1RetryWithNewerEpochErrors), supportedOperation) val expectedEpochErrors = topicPartitions.map(_ -> Errors.INVALID_PRODUCER_EPOCH).toMap assertEquals(expectedEpochErrors, transaction1RetryWithSameEpochErrors) // Trying to add more transactional data for the same transactional ID and producer ID, but an older epoch should immediately return with error and keep the old data queued to send. - addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1RetryWithOldEpochErrors)) + addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1RetryWithOldEpochErrors), supportedOperation) assertEquals(expectedEpochErrors, transaction1RetryWithOldEpochErrors) val requestsAndHandlers = addPartitionsToTxnManager.generateRequests().asScala @@ -167,33 +154,14 @@ class AddPartitionsToTxnManagerTest { when(partitionFor.apply(transactionalId1)).thenReturn(0) when(partitionFor.apply(transactionalId2)).thenReturn(1) when(partitionFor.apply(transactionalId3)).thenReturn(2) - when(metadataCache.getTopicMetadata(Set(Topic.TRANSACTION_STATE_TOPIC_NAME), config.interBrokerListenerName)) - .thenReturn(Seq( - new MetadataResponseData.MetadataResponseTopic() - .setName(Topic.TRANSACTION_STATE_TOPIC_NAME) - .setPartitions(List( - new MetadataResponseData.MetadataResponsePartition() - .setPartitionIndex(0) - .setLeaderId(0), - new MetadataResponseData.MetadataResponsePartition() - .setPartitionIndex(1) - .setLeaderId(1), - new MetadataResponseData.MetadataResponsePartition() - .setPartitionIndex(2) - .setLeaderId(2) - ).asJava) - )) - when(metadataCache.getAliveBrokerNode(0, config.interBrokerListenerName)) - .thenReturn(Some(node0)) - when(metadataCache.getAliveBrokerNode(1, config.interBrokerListenerName)) - .thenReturn(Some(node1)) - when(metadataCache.getAliveBrokerNode(2, config.interBrokerListenerName)) - .thenReturn(Some(node2)) + mockTransactionStateMetadata(0, 0, Some(node0)) + mockTransactionStateMetadata(1, 1, Some(node1)) + mockTransactionStateMetadata(2, 2, Some(node2)) val transactionErrors = mutable.Map[TopicPartition, Errors]() - addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transactionErrors)) - addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transactionErrors)) + addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), supportedOperation) + addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), supportedOperation) val requestsAndHandlers = addPartitionsToTxnManager.generateRequests().asScala assertEquals(2, requestsAndHandlers.size) @@ -206,8 +174,8 @@ class AddPartitionsToTxnManagerTest { } } - addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transactionErrors)) - addPartitionsToTxnManager.verifyTransaction(transactionalId3, producerId3, producerEpoch = 0, topicPartitions, setErrors(transactionErrors)) + addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), supportedOperation) + addPartitionsToTxnManager.verifyTransaction(transactionalId3, producerId3, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), supportedOperation) // Test creationTimeMs increases too. time.sleep(10) @@ -240,58 +208,24 @@ class AddPartitionsToTxnManagerTest { producerId1, producerEpoch = 0, topicPartitions, - setErrors(errors) + setErrors(errors), + supportedOperation ) assertEquals(topicPartitions.map(tp => tp -> Errors.COORDINATOR_NOT_AVAILABLE).toMap, errors) } // The transaction state topic does not exist. - when(metadataCache.getTopicMetadata(Set(Topic.TRANSACTION_STATE_TOPIC_NAME), config.interBrokerListenerName)) - .thenReturn(Seq()) - checkError() - - // The metadata of the transaction state topic returns an error. - when(metadataCache.getTopicMetadata(Set(Topic.TRANSACTION_STATE_TOPIC_NAME), config.interBrokerListenerName)) - .thenReturn(Seq( - new MetadataResponseData.MetadataResponseTopic() - .setName(Topic.TRANSACTION_STATE_TOPIC_NAME) - .setErrorCode(Errors.BROKER_NOT_AVAILABLE.code) - )) - checkError() - - // The partition does not exist. - when(metadataCache.getTopicMetadata(Set(Topic.TRANSACTION_STATE_TOPIC_NAME), config.interBrokerListenerName)) - .thenReturn(Seq( - new MetadataResponseData.MetadataResponseTopic() - .setName(Topic.TRANSACTION_STATE_TOPIC_NAME) - )) + when(metadataCache.getPartitionInfo(Topic.TRANSACTION_STATE_TOPIC_NAME, 0)) + .thenReturn(Option.empty) checkError() // The partition has no leader. - when(metadataCache.getTopicMetadata(Set(Topic.TRANSACTION_STATE_TOPIC_NAME), config.interBrokerListenerName)) - .thenReturn(Seq( - new MetadataResponseData.MetadataResponseTopic() - .setName(Topic.TRANSACTION_STATE_TOPIC_NAME) - .setPartitions(List( - new MetadataResponseData.MetadataResponsePartition() - .setPartitionIndex(0) - .setLeaderId(-1) - ).asJava) - )) + mockTransactionStateMetadata(0, -1, Option.empty) checkError() // The leader is not available. - when(metadataCache.getTopicMetadata(Set(Topic.TRANSACTION_STATE_TOPIC_NAME), config.interBrokerListenerName)) - .thenReturn(Seq( - new MetadataResponseData.MetadataResponseTopic() - .setName(Topic.TRANSACTION_STATE_TOPIC_NAME) - .setPartitions(List( - new MetadataResponseData.MetadataResponsePartition() - .setPartitionIndex(0) - .setLeaderId(0) - ).asJava) - )) + mockTransactionStateMetadata(0, 0, Option.empty) checkError() } @@ -299,18 +233,7 @@ class AddPartitionsToTxnManagerTest { def testAddPartitionsToTxnHandlerErrorHandling(): Unit = { when(partitionFor.apply(transactionalId1)).thenReturn(0) when(partitionFor.apply(transactionalId2)).thenReturn(0) - when(metadataCache.getTopicMetadata(Set(Topic.TRANSACTION_STATE_TOPIC_NAME), config.interBrokerListenerName)) - .thenReturn(Seq( - new MetadataResponseData.MetadataResponseTopic() - .setName(Topic.TRANSACTION_STATE_TOPIC_NAME) - .setPartitions(List( - new MetadataResponseData.MetadataResponsePartition() - .setPartitionIndex(0) - .setLeaderId(0) - ).asJava) - )) - when(metadataCache.getAliveBrokerNode(0, config.interBrokerListenerName)) - .thenReturn(Some(node0)) + mockTransactionStateMetadata(0, 0, Some(node0)) val transaction1Errors = mutable.Map[TopicPartition, Errors]() val transaction2Errors = mutable.Map[TopicPartition, Errors]() @@ -319,8 +242,16 @@ class AddPartitionsToTxnManagerTest { transaction1Errors.clear() transaction2Errors.clear() - addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1Errors)) - addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transaction2Errors)) + addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1Errors), supportedOperation) + addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transaction2Errors), supportedOperation) + } + + def addTransactionsToVerifyRequestVersion(operationExpected: SupportedOperation): Unit = { + transaction1Errors.clear() + transaction2Errors.clear() + + addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1Errors), operationExpected) + addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transaction2Errors), operationExpected) } val expectedAuthErrors = topicPartitions.map(_ -> Errors.SASL_AUTHENTICATION_FAILED).toMap @@ -367,6 +298,32 @@ class AddPartitionsToTxnManagerTest { receiveResponse(mixedErrorsResponse) assertEquals(expectedTransaction1Errors, transaction1Errors) assertEquals(expectedTransaction2Errors, transaction2Errors) + + val preConvertedAbortableTransaction1Errors = topicPartitions.map(_ -> Errors.ABORTABLE_TRANSACTION).toMap + val preConvertedAbortableTransaction2Errors = Map(new TopicPartition("foo", 1) -> Errors.NONE, + new TopicPartition("foo", 2) -> Errors.ABORTABLE_TRANSACTION, + new TopicPartition("foo", 3) -> Errors.NONE) + val abortableTransaction1ErrorResponse = AddPartitionsToTxnResponse.resultForTransaction(transactionalId1, preConvertedAbortableTransaction1Errors.asJava) + val abortableTransaction2ErrorResponse = AddPartitionsToTxnResponse.resultForTransaction(transactionalId2, preConvertedAbortableTransaction2Errors.asJava) + val mixedErrorsAddPartitionsResponseAbortableError = new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData() + .setResultsByTransaction(new AddPartitionsToTxnResultCollection(Seq(abortableTransaction1ErrorResponse, abortableTransaction2ErrorResponse).iterator.asJava))) + val mixedAbortableErrorsResponse = clientResponse(mixedErrorsAddPartitionsResponseAbortableError) + + val expectedAbortableTransaction1ErrorsLowerVersion = topicPartitions.map(_ -> Errors.INVALID_TXN_STATE).toMap + val expectedAbortableTransaction2ErrorsLowerVersion = Map(new TopicPartition("foo", 2) -> Errors.INVALID_TXN_STATE) + + val expectedAbortableTransaction1ErrorsHigherVersion = topicPartitions.map(_ -> Errors.ABORTABLE_TRANSACTION).toMap + val expectedAbortableTransaction2ErrorsHigherVersion = Map(new TopicPartition("foo", 2) -> Errors.ABORTABLE_TRANSACTION) + + addTransactionsToVerifyRequestVersion(defaultError) + receiveResponse(mixedAbortableErrorsResponse) + assertEquals(expectedAbortableTransaction1ErrorsLowerVersion, transaction1Errors) + assertEquals(expectedAbortableTransaction2ErrorsLowerVersion, transaction2Errors) + + addTransactionsToVerifyRequestVersion(genericError) + receiveResponse(mixedAbortableErrorsResponse) + assertEquals(expectedAbortableTransaction1ErrorsHigherVersion, transaction1Errors) + assertEquals(expectedAbortableTransaction2ErrorsHigherVersion, transaction2Errors) } @Test @@ -380,23 +337,8 @@ class AddPartitionsToTxnManagerTest { when(partitionFor.apply(transactionalId1)).thenReturn(0) when(partitionFor.apply(transactionalId2)).thenReturn(1) - when(metadataCache.getTopicMetadata(Set(Topic.TRANSACTION_STATE_TOPIC_NAME), config.interBrokerListenerName)) - .thenReturn(Seq( - new MetadataResponseData.MetadataResponseTopic() - .setName(Topic.TRANSACTION_STATE_TOPIC_NAME) - .setPartitions(List( - new MetadataResponseData.MetadataResponsePartition() - .setPartitionIndex(0) - .setLeaderId(0), - new MetadataResponseData.MetadataResponsePartition() - .setPartitionIndex(1) - .setLeaderId(1) - ).asJava) - )) - when(metadataCache.getAliveBrokerNode(0, config.interBrokerListenerName)) - .thenReturn(Some(node0)) - when(metadataCache.getAliveBrokerNode(1, config.interBrokerListenerName)) - .thenReturn(Some(node1)) + mockTransactionStateMetadata(0, 0, Some(node0)) + mockTransactionStateMetadata(1, 1, Some(node1)) // Update max verification time when we see a higher verification time. when(mockVerificationTime.update(anyLong())).thenAnswer { invocation => @@ -419,8 +361,8 @@ class AddPartitionsToTxnManagerTest { ) try { - addPartitionsManagerWithMockedMetrics.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transactionErrors)) - addPartitionsManagerWithMockedMetrics.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transactionErrors)) + addPartitionsManagerWithMockedMetrics.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), supportedOperation) + addPartitionsManagerWithMockedMetrics.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), supportedOperation) time.sleep(100) @@ -458,6 +400,19 @@ class AddPartitionsToTxnManagerTest { } } + private def mockTransactionStateMetadata(partitionIndex: Int, leaderId: Int, leaderNode: Option[Node]): Unit = { + when(metadataCache.getPartitionInfo(Topic.TRANSACTION_STATE_TOPIC_NAME, partitionIndex)) + .thenReturn(Some( + new UpdateMetadataPartitionState() + .setTopicName(Topic.TRANSACTION_STATE_TOPIC_NAME) + .setPartitionIndex(partitionIndex) + .setLeader(leaderId))) + if (leaderId != MetadataResponse.NO_LEADER_ID) { + when(metadataCache.getAliveBrokerNode(leaderId, config.interBrokerListenerName)) + .thenReturn(leaderNode) + } + } + private def clientResponse( response: AbstractResponse, authException: AuthenticationException = null, diff --git a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala index 6e296c2892..bc3da1852a 100644 --- a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala @@ -163,7 +163,7 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest { val verifyErrors = verifyResponse.errors() - assertEquals(Collections.singletonMap(transactionalId, Collections.singletonMap(tp0, Errors.INVALID_TXN_STATE)), verifyErrors) + assertEquals(Collections.singletonMap(transactionalId, Collections.singletonMap(tp0, Errors.ABORTABLE_TRANSACTION)), verifyErrors) } private def setUpTransactions(transactionalId: String, verifyOnly: Boolean, partitions: Set[TopicPartition]): (Int, AddPartitionsToTxnTransaction) = { diff --git a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala index d7925ef8b5..a36813986e 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala @@ -197,11 +197,18 @@ class BrokerLifecycleManagerTest { result } - def poll[T](context: RegistrationTestContext, manager: BrokerLifecycleManager, future: Future[T]): T = { - while (!future.isDone || context.mockClient.hasInFlightRequests) { - context.poll() + def poll[T](ctx: RegistrationTestContext, manager: BrokerLifecycleManager, future: Future[T]): T = { + while (ctx.mockChannelManager.unsentQueue.isEmpty) { + // Cancel a potential timeout event, so it doesn't get activated if we advance the clock too much + manager.eventQueue.cancelDeferred("initialRegistrationTimeout") + + // If the manager is idling until scheduled events we need to advance the clock + if (manager.eventQueue.firstDeferredIfIdling().isPresent) + ctx.time.sleep(5) manager.eventQueue.wakeup() - context.time.sleep(5) + } + while (!future.isDone) { + ctx.poll() } future.get } @@ -219,15 +226,16 @@ class BrokerLifecycleManagerTest { Collections.emptyMap(), OptionalLong.empty()) poll(ctx, manager, registration) + def nextHeartbeatDirs(): Set[String] = + poll(ctx, manager, prepareResponse[BrokerHeartbeatRequest](ctx, new BrokerHeartbeatResponse(new BrokerHeartbeatResponseData()))) + .data().offlineLogDirs().asScala.map(_.toString).toSet + assertEquals(Set.empty, nextHeartbeatDirs()) manager.propagateDirectoryFailure(Uuid.fromString("h3sC4Yk-Q9-fd0ntJTocCA")) + assertEquals(Set("h3sC4Yk-Q9-fd0ntJTocCA"), nextHeartbeatDirs()) manager.propagateDirectoryFailure(Uuid.fromString("ej8Q9_d2Ri6FXNiTxKFiow")) + assertEquals(Set("h3sC4Yk-Q9-fd0ntJTocCA", "ej8Q9_d2Ri6FXNiTxKFiow"), nextHeartbeatDirs()) manager.propagateDirectoryFailure(Uuid.fromString("1iF76HVNRPqC7Y4r6647eg")) - val latestHeartbeat = Seq.fill(10)( - prepareResponse[BrokerHeartbeatRequest](ctx, new BrokerHeartbeatResponse(new BrokerHeartbeatResponseData())) - ).map(poll(ctx, manager, _)).last - assertEquals( - Set("h3sC4Yk-Q9-fd0ntJTocCA", "ej8Q9_d2Ri6FXNiTxKFiow", "1iF76HVNRPqC7Y4r6647eg").map(Uuid.fromString), - latestHeartbeat.data().offlineLogDirs().asScala.toSet) + assertEquals(Set("h3sC4Yk-Q9-fd0ntJTocCA", "ej8Q9_d2Ri6FXNiTxKFiow", "1iF76HVNRPqC7Y4r6647eg"), nextHeartbeatDirs()) manager.close() } @@ -254,33 +262,35 @@ class BrokerLifecycleManagerTest { @Test def testKraftJBODMetadataVersionUpdateEvent(): Unit = { - val context = new RegistrationTestContext(configProperties) - val manager = new BrokerLifecycleManager(context.config, context.time, "successful-registration-", isZkBroker = false, Set(Uuid.fromString("gCpDJgRlS2CBCpxoP2VMsQ"))) + val ctx = new RegistrationTestContext(configProperties) + val manager = new BrokerLifecycleManager(ctx.config, ctx.time, "jbod-metadata-version-update", isZkBroker = false, Set(Uuid.fromString("gCpDJgRlS2CBCpxoP2VMsQ"))) val controllerNode = new Node(3000, "localhost", 8021) - context.controllerNodeProvider.node.set(controllerNode) - manager.start(() => context.highestMetadataOffset.get(), - context.mockChannelManager, context.clusterId, context.advertisedListeners, + ctx.controllerNodeProvider.node.set(controllerNode) + + manager.start(() => ctx.highestMetadataOffset.get(), + ctx.mockChannelManager, ctx.clusterId, ctx.advertisedListeners, Collections.emptyMap(), OptionalLong.of(10L)) - TestUtils.retry(60000) { - assertEquals(1, context.mockChannelManager.unsentQueue.size) - assertEquals(10L, context.mockChannelManager.unsentQueue.getFirst.request.build().asInstanceOf[BrokerRegistrationRequest].data().previousBrokerEpoch()) - } - context.mockClient.prepareResponseFrom(new BrokerRegistrationResponse( - new BrokerRegistrationResponseData().setBrokerEpoch(1000)), controllerNode) - TestUtils.retry(10000) { - context.poll() - assertEquals(1000L, manager.brokerEpoch) - } + def doPoll[T<:AbstractRequest](response: AbstractResponse) = poll(ctx, manager, prepareResponse[T](ctx, response)) + def nextHeartbeatRequest() = doPoll[AbstractRequest](new BrokerHeartbeatResponse(new BrokerHeartbeatResponseData())) + def nextRegistrationRequest(epoch: Long) = + doPoll[BrokerRegistrationRequest](new BrokerRegistrationResponse(new BrokerRegistrationResponseData().setBrokerEpoch(epoch))) + + // Broker registers and response sets epoch to 1000L + assertEquals(10L, nextRegistrationRequest(1000L).data().previousBrokerEpoch()) + + nextHeartbeatRequest() // poll for next request as way to synchronize with the new value into brokerEpoch + assertEquals(1000L, manager.brokerEpoch) + + // Trigger JBOD MV update manager.handleKraftJBODMetadataVersionUpdate() - context.mockClient.prepareResponseFrom(new BrokerRegistrationResponse( - new BrokerRegistrationResponseData().setBrokerEpoch(1200)), controllerNode) - TestUtils.retry(60000) { - context.time.sleep(100) - context.poll() - manager.eventQueue.wakeup() - assertEquals(1200, manager.brokerEpoch) - } + + // Accept new registration, response sets epoch to 1200 + nextRegistrationRequest(1200L) + + nextHeartbeatRequest() + assertEquals(1200L, manager.brokerEpoch) + manager.close() } } diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index c6d5b026f4..9173af4f2e 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -55,6 +55,7 @@ import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer} import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion, ProducerIdsBlock} import org.apache.kafka.server.util.FutureUtils +import org.apache.kafka.storage.internals.log.CleanerConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.junit.jupiter.params.ParameterizedTest @@ -318,19 +319,19 @@ class ControllerApisTest { setResourceName("1"). setResourceType(ConfigResource.Type.BROKER.id()). setConfigs(new OldAlterableConfigCollection(util.Arrays.asList(new OldAlterableConfig(). - setName(KafkaConfig.LogCleanerBackoffMsProp). + setName(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP). setValue("100000")).iterator())), new OldAlterConfigsResource(). setResourceName("2"). setResourceType(ConfigResource.Type.BROKER.id()). setConfigs(new OldAlterableConfigCollection(util.Arrays.asList(new OldAlterableConfig(). - setName(KafkaConfig.LogCleanerBackoffMsProp). + setName(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP). setValue("100000")).iterator())), new OldAlterConfigsResource(). setResourceName("2"). setResourceType(ConfigResource.Type.BROKER.id()). setConfigs(new OldAlterableConfigCollection(util.Arrays.asList(new OldAlterableConfig(). - setName(KafkaConfig.LogCleanerBackoffMsProp). + setName(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP). setValue("100000")).iterator())), new OldAlterConfigsResource(). setResourceName("baz"). @@ -472,7 +473,7 @@ class ControllerApisTest { setResourceName("1"). setResourceType(ConfigResource.Type.BROKER.id()). setConfigs(new AlterableConfigCollection(util.Arrays.asList(new AlterableConfig(). - setName(KafkaConfig.LogCleanerBackoffMsProp). + setName(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP). setValue("100000"). setConfigOperation(AlterConfigOp.OpType.SET.id())).iterator())), new AlterConfigsResource(). @@ -536,14 +537,14 @@ class ControllerApisTest { setResourceName("3"). setResourceType(ConfigResource.Type.BROKER.id()). setConfigs(new AlterableConfigCollection(util.Arrays.asList(new AlterableConfig(). - setName(KafkaConfig.LogCleanerBackoffMsProp). + setName(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP). setValue("100000"). setConfigOperation(AlterConfigOp.OpType.SET.id())).iterator())), new AlterConfigsResource(). setResourceName("3"). setResourceType(ConfigResource.Type.BROKER.id()). setConfigs(new AlterableConfigCollection(util.Arrays.asList(new AlterableConfig(). - setName(KafkaConfig.LogCleanerBackoffMsProp). + setName(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP). setValue("100000"). setConfigOperation(AlterConfigOp.OpType.SET.id())).iterator())), new AlterConfigsResource(). diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 1163076d2c..fca0f6ce66 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -39,7 +39,7 @@ import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.util.KafkaScheduler -import org.apache.kafka.storage.internals.log.{LogConfig, ProducerStateManagerConfig} +import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, ProducerStateManagerConfig} import org.apache.kafka.test.MockMetricsReporter import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -216,7 +216,7 @@ class DynamicBrokerConfigTest { verifyConfigUpdateWithInvalidConfig(config, origProps, validProps, nonDynamicProps) // Test update of configs with invalid type - val invalidProps = Map(KafkaConfig.LogCleanerThreadsProp -> "invalid") + val invalidProps = Map(CleanerConfig.LOG_CLEANER_THREADS_PROP -> "invalid") verifyConfigUpdateWithInvalidConfig(config, origProps, validProps, invalidProps) val excludedTopicConfig = Map(KafkaConfig.LogMessageFormatVersionProp -> "0.10.2") @@ -226,21 +226,21 @@ class DynamicBrokerConfigTest { @Test def testConfigUpdateWithReconfigurableValidationFailure(): Unit = { val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - origProps.put(KafkaConfig.LogCleanerDedupeBufferSizeProp, "100000000") + origProps.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "100000000") val config = KafkaConfig(origProps) config.dynamicConfig.initialize(None, None) val validProps = Map.empty[String, String] - val invalidProps = Map(KafkaConfig.LogCleanerThreadsProp -> "20") + val invalidProps = Map(CleanerConfig.LOG_CLEANER_THREADS_PROP -> "20") def validateLogCleanerConfig(configs: util.Map[String, _]): Unit = { - val cleanerThreads = configs.get(KafkaConfig.LogCleanerThreadsProp).toString.toInt + val cleanerThreads = configs.get(CleanerConfig.LOG_CLEANER_THREADS_PROP).toString.toInt if (cleanerThreads <=0 || cleanerThreads >= 5) throw new ConfigException(s"Invalid cleaner threads $cleanerThreads") } val reconfigurable = new Reconfigurable { override def configure(configs: util.Map[String, _]): Unit = {} - override def reconfigurableConfigs(): util.Set[String] = Set(KafkaConfig.LogCleanerThreadsProp).asJava + override def reconfigurableConfigs(): util.Set[String] = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP).asJava override def validateReconfiguration(configs: util.Map[String, _]): Unit = validateLogCleanerConfig(configs) override def reconfigure(configs: util.Map[String, _]): Unit = {} } @@ -249,7 +249,7 @@ class DynamicBrokerConfigTest { config.dynamicConfig.removeReconfigurable(reconfigurable) val brokerReconfigurable = new BrokerReconfigurable { - override def reconfigurableConfigs: collection.Set[String] = Set(KafkaConfig.LogCleanerThreadsProp) + override def reconfigurableConfigs: collection.Set[String] = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP) override def validateReconfiguration(newConfig: KafkaConfig): Unit = validateLogCleanerConfig(newConfig.originals) override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = {} } @@ -261,8 +261,8 @@ class DynamicBrokerConfigTest { def testReconfigurableValidation(): Unit = { val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val config = KafkaConfig(origProps) - val invalidReconfigurableProps = Set(KafkaConfig.LogCleanerThreadsProp, KafkaConfig.BrokerIdProp, "some.prop") - val validReconfigurableProps = Set(KafkaConfig.LogCleanerThreadsProp, KafkaConfig.LogCleanerDedupeBufferSizeProp, "some.prop") + val invalidReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, KafkaConfig.BrokerIdProp, "some.prop") + val validReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "some.prop") def createReconfigurable(configs: Set[String]) = new Reconfigurable { override def configure(configs: util.Map[String, _]): Unit = {} diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index adcc5e824c..756b4e0bac 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -2487,6 +2487,7 @@ class KafkaApisTest extends Logging { responseCallback.capture(), any(), any(), + any(), any() )).thenAnswer(_ => responseCallback.getValue.apply(Map(tp -> new PartitionResponse(Errors.INVALID_PRODUCER_EPOCH)))) @@ -2548,6 +2549,7 @@ class KafkaApisTest extends Logging { responseCallback.capture(), any(), any(), + any(), any()) ).thenAnswer(_ => responseCallback.getValue.apply(Map(tp -> new PartitionResponse(Errors.NOT_LEADER_OR_FOLLOWER)))) @@ -2612,6 +2614,7 @@ class KafkaApisTest extends Logging { responseCallback.capture(), any(), any(), + any(), any()) ).thenAnswer(_ => responseCallback.getValue.apply(Map(tp -> new PartitionResponse(Errors.NOT_LEADER_OR_FOLLOWER)))) @@ -2675,6 +2678,7 @@ class KafkaApisTest extends Logging { responseCallback.capture(), any(), any(), + any(), any()) ).thenAnswer(_ => responseCallback.getValue.apply(Map(tp -> new PartitionResponse(Errors.NOT_LEADER_OR_FOLLOWER)))) @@ -2742,6 +2746,7 @@ class KafkaApisTest extends Logging { any(), any(), any(), + any(), any()) } finally { kafkaApis.close() diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 59a5e3a5c3..e535af50bd 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -40,7 +40,7 @@ import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_8_2, IBP_3_0_IV1} import org.apache.kafka.server.config.ServerTopicConfigSynonyms import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig -import org.apache.kafka.storage.internals.log.LogConfig +import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import org.junit.jupiter.api.function.Executable import scala.annotation.nowarn @@ -845,14 +845,14 @@ class KafkaConfigTest { case KafkaConfig.LogRetentionBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KafkaConfig.LogCleanupIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case KafkaConfig.LogCleanupPolicyProp => assertPropertyInvalid(baseProperties, name, "unknown_policy", "0") - case KafkaConfig.LogCleanerIoMaxBytesPerSecondProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.LogCleanerDedupeBufferSizeProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "1024") - case KafkaConfig.LogCleanerDedupeBufferLoadFactorProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.LogCleanerEnableProp => assertPropertyInvalid(baseProperties, name, "not_a_boolean") - case KafkaConfig.LogCleanerDeleteRetentionMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.LogCleanerMinCompactionLagMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.LogCleanerMaxCompactionLagMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.LogCleanerMinCleanRatioProp => assertPropertyInvalid(baseProperties, name, "not_a_number") + case CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number") + case CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "1024") + case CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number") + case CleanerConfig.LOG_CLEANER_ENABLE_PROP => assertPropertyInvalid(baseProperties, name, "not_a_boolean") + case CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number") + case CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number") + case CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number") + case CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number") case KafkaConfig.LogIndexSizeMaxBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "3") case KafkaConfig.LogFlushIntervalMessagesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case KafkaConfig.LogFlushSchedulerIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") diff --git a/core/src/test/scala/unit/kafka/server/KafkaServerTest.scala b/core/src/test/scala/unit/kafka/server/KafkaServerTest.scala index 1372690afe..bf6312f863 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaServerTest.scala @@ -17,12 +17,13 @@ package kafka.server -import kafka.utils.TestUtils +import kafka.utils.{CoreUtils, TestUtils} import org.apache.kafka.common.security.JaasUtils import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertThrows, fail} import org.junit.jupiter.api.Test -import java.util.Properties +import java.util.Properties +import java.net.{InetAddress, ServerSocket} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig @@ -42,6 +43,24 @@ class KafkaServerTest extends QuorumTestHarness { TestUtils.shutdownServers(Seq(server1, server2)) } + @Test + def testListenerPortAlreadyInUse(): Unit = { + val serverSocket = new ServerSocket(0, 0, InetAddress.getLoopbackAddress) + + var kafkaServer : Option[KafkaServer] = None + try { + TestUtils.waitUntilTrue(() => serverSocket.isBound, "Server socket failed to bind.") + // start a server with listener on the port already bound + assertThrows(classOf[RuntimeException], + () => kafkaServer = Option(createServerWithListenerOnPort(serverSocket.getLocalPort)), + "Expected RuntimeException due to address already in use during KafkaServer startup" + ) + } finally { + CoreUtils.swallow(serverSocket.close(), this); + TestUtils.shutdownServers(kafkaServer.toList) + } + } + @Test def testCreatesProperZkConfigWhenSaslDisabled(): Unit = { val props = new Properties @@ -161,4 +180,11 @@ class KafkaServerTest extends QuorumTestHarness { TestUtils.createServer(kafkaConfig) } + def createServerWithListenerOnPort(port: Int): KafkaServer = { + val props = TestUtils.createBrokerConfig(0, zkConnect) + props.put(KafkaConfig.ListenersProp, s"PLAINTEXT://localhost:$port") + val kafkaConfig = KafkaConfig.fromProps(props) + TestUtils.createServer(kafkaConfig) + } + } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 79cfcd597c..f2a277035a 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -114,6 +114,7 @@ class ReplicaManagerTest { protected var mockRemoteLogManager: RemoteLogManager = _ protected var addPartitionsToTxnManager: AddPartitionsToTxnManager = _ protected var brokerTopicStats: BrokerTopicStats = _ + private val supportedOperation = genericError // Constants defined for readability protected val zkVersion = 0 @@ -134,7 +135,7 @@ class ReplicaManagerTest { addPartitionsToTxnManager = mock(classOf[AddPartitionsToTxnManager]) // Anytime we try to verify, just automatically run the callback as though the transaction was verified. - when(addPartitionsToTxnManager.verifyTransaction(any(), any(), any(), any(), any())).thenAnswer { invocationOnMock => + when(addPartitionsToTxnManager.verifyTransaction(any(), any(), any(), any(), any(), any())).thenAnswer { invocationOnMock => val callback = invocationOnMock.getArgument(4, classOf[AddPartitionsToTxnManager.AppendCallback]) callback(Map.empty[TopicPartition, Errors].toMap) } @@ -2184,7 +2185,7 @@ class ReplicaManagerTest { val idempotentRecords = MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, producerEpoch, sequence, new SimpleRecord("message".getBytes)) handleProduceAppend(replicaManager, tp0, idempotentRecords, transactionalId = null) - verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any[AddPartitionsToTxnManager.AppendCallback]()) + verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any[AddPartitionsToTxnManager.AppendCallback](), any()) assertEquals(VerificationGuard.SENTINEL, getVerificationGuard(replicaManager, tp0, producerId)) // If we supply a transactional ID and some transactional and some idempotent records, we should only verify the topic partition with transactional records. @@ -2199,7 +2200,8 @@ class ReplicaManagerTest { ArgumentMatchers.eq(producerId), ArgumentMatchers.eq(producerEpoch), ArgumentMatchers.eq(Seq(tp0)), - any[AddPartitionsToTxnManager.AppendCallback]() + any[AddPartitionsToTxnManager.AppendCallback](), + any() ) assertNotEquals(VerificationGuard.SENTINEL, getVerificationGuard(replicaManager, tp0, producerId)) assertEquals(VerificationGuard.SENTINEL, getVerificationGuard(replicaManager, tp1, producerId)) @@ -2235,7 +2237,8 @@ class ReplicaManagerTest { ArgumentMatchers.eq(producerId), ArgumentMatchers.eq(producerEpoch), ArgumentMatchers.eq(Seq(tp0)), - appendCallback.capture() + appendCallback.capture(), + any() ) val verificationGuard = getVerificationGuard(replicaManager, tp0, producerId) assertEquals(verificationGuard, getVerificationGuard(replicaManager, tp0, producerId)) @@ -2254,7 +2257,8 @@ class ReplicaManagerTest { ArgumentMatchers.eq(producerId), ArgumentMatchers.eq(producerEpoch), ArgumentMatchers.eq(Seq(tp0)), - appendCallback2.capture() + appendCallback2.capture(), + any() ) assertEquals(verificationGuard, getVerificationGuard(replicaManager, tp0, producerId)) @@ -2293,7 +2297,8 @@ class ReplicaManagerTest { ArgumentMatchers.eq(producerId), ArgumentMatchers.eq(producerEpoch), ArgumentMatchers.eq(Seq(tp0)), - appendCallback.capture() + appendCallback.capture(), + any() ) val verificationGuard = getVerificationGuard(replicaManager, tp0, producerId) assertEquals(verificationGuard, getVerificationGuard(replicaManager, tp0, producerId)) @@ -2315,7 +2320,8 @@ class ReplicaManagerTest { ArgumentMatchers.eq(producerId), ArgumentMatchers.eq(producerEpoch), ArgumentMatchers.eq(Seq(tp0)), - appendCallback2.capture() + appendCallback2.capture(), + any() ) assertEquals(verificationGuard, getVerificationGuard(replicaManager, tp0, producerId)) @@ -2393,7 +2399,7 @@ class ReplicaManagerTest { assertThrows(classOf[InvalidPidMappingException], () => handleProduceAppendToMultipleTopics(replicaManager, transactionalRecords, transactionalId = transactionalId)) // We should not add these partitions to the manager to verify. - verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any()) + verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any(), any()) } finally { replicaManager.shutdown(checkpointHW = false) } @@ -2417,7 +2423,7 @@ class ReplicaManagerTest { handleProduceAppend(replicaManager, tp0, transactionalRecords, transactionalId = transactionalId).onFire { response => assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, response.error) } - verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any()) + verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any(), any()) } finally { replicaManager.shutdown(checkpointHW = false) } @@ -2451,7 +2457,7 @@ class ReplicaManagerTest { assertEquals(VerificationGuard.SENTINEL, getVerificationGuard(replicaManager, tp, producerId)) // We should not add these partitions to the manager to verify. - verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any()) + verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any(), any()) // Dynamically enable verification. config.dynamicConfig.initialize(None, None) @@ -2465,7 +2471,7 @@ class ReplicaManagerTest { new SimpleRecord("message".getBytes)) handleProduceAppend(replicaManager, tp, moreTransactionalRecords, transactionalId = transactionalId) - verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any()) + verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any(), any()) assertEquals(VerificationGuard.SENTINEL, getVerificationGuard(replicaManager, tp, producerId)) assertTrue(replicaManager.localLog(tp).get.hasOngoingTransaction(producerId)) } finally { @@ -2499,7 +2505,8 @@ class ReplicaManagerTest { ArgumentMatchers.eq(producerId), ArgumentMatchers.eq(producerEpoch), ArgumentMatchers.eq(Seq(tp0)), - appendCallback.capture() + appendCallback.capture(), + any() ) val verificationGuard = getVerificationGuard(replicaManager, tp0, producerId) assertEquals(verificationGuard, getVerificationGuard(replicaManager, tp0, producerId)) @@ -2519,7 +2526,7 @@ class ReplicaManagerTest { // This time we do not verify handleProduceAppend(replicaManager, tp0, transactionalRecords, transactionalId = transactionalId) - verify(addPartitionsToTxnManager, times(1)).verifyTransaction(any(), any(), any(), any(), any()) + verify(addPartitionsToTxnManager, times(1)).verifyTransaction(any(), any(), any(), any(), any(), any()) assertEquals(VerificationGuard.SENTINEL, getVerificationGuard(replicaManager, tp0, producerId)) assertTrue(replicaManager.localLog(tp0).get.hasOngoingTransaction(producerId)) } finally { @@ -2528,7 +2535,16 @@ class ReplicaManagerTest { } @ParameterizedTest - @EnumSource(value = classOf[Errors], names = Array("NOT_COORDINATOR", "CONCURRENT_TRANSACTIONS", "COORDINATOR_LOAD_IN_PROGRESS", "COORDINATOR_NOT_AVAILABLE")) + @EnumSource( + value = classOf[Errors], + names = Array( + "NOT_COORDINATOR", + "CONCURRENT_TRANSACTIONS", + "NETWORK_EXCEPTION", + "COORDINATOR_LOAD_IN_PROGRESS", + "COORDINATOR_NOT_AVAILABLE" + ) + ) def testVerificationErrorConversions(error: Errors): Unit = { val tp0 = new TopicPartition(topic, 0) val producerId = 24L @@ -2554,7 +2570,8 @@ class ReplicaManagerTest { ArgumentMatchers.eq(producerId), ArgumentMatchers.eq(producerEpoch), ArgumentMatchers.eq(Seq(tp0)), - appendCallback.capture() + appendCallback.capture(), + any() ) // Confirm we did not write to the log and instead returned the converted error with the correct error message. @@ -2584,7 +2601,8 @@ class ReplicaManagerTest { ArgumentMatchers.eq(producerId), ArgumentMatchers.eq(producerEpoch), ArgumentMatchers.eq(Seq(tp0)), - appendCallback.capture() + appendCallback.capture(), + any() ) assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, result.assertFired.left.getOrElse(Errors.NONE)) } finally { @@ -3042,6 +3060,7 @@ class ReplicaManagerTest { transactionalId = transactionalId, entriesPerPartition = entriesToAppend, responseCallback = appendCallback, + supportedOperation = supportedOperation ) result @@ -3069,6 +3088,7 @@ class ReplicaManagerTest { transactionalId = transactionalId, entriesPerPartition = entriesPerPartition, responseCallback = appendCallback, + supportedOperation = supportedOperation ) result @@ -3093,7 +3113,8 @@ class ReplicaManagerTest { producerId, producerEpoch, baseSequence, - postVerificationCallback + postVerificationCallback, + supportedOperation ) result } diff --git a/core/src/test/scala/unit/kafka/server/streamaspect/ElasticReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/streamaspect/ElasticReplicaManagerTest.scala index b9d4572df5..895b21a643 100644 --- a/core/src/test/scala/unit/kafka/server/streamaspect/ElasticReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/streamaspect/ElasticReplicaManagerTest.scala @@ -59,7 +59,7 @@ class ElasticReplicaManagerTest extends ReplicaManagerTest { addPartitionsToTxnManager = mock(classOf[AddPartitionsToTxnManager]) // Anytime we try to verify, just automatically run the callback as though the transaction was verified. - when(addPartitionsToTxnManager.verifyTransaction(any(), any(), any(), any(), any())).thenAnswer { invocationOnMock => + when(addPartitionsToTxnManager.verifyTransaction(any(), any(), any(), any(), any(), any())).thenAnswer { invocationOnMock => val callback = invocationOnMock.getArgument(4, classOf[AddPartitionsToTxnManager.AppendCallback]) callback(Map.empty[TopicPartition, Errors].toMap) } diff --git a/core/src/test/scala/unit/kafka/utils/ShutdownableThreadTest.scala b/core/src/test/scala/unit/kafka/utils/ShutdownableThreadTest.scala index 7b086d1150..da998903d2 100644 --- a/core/src/test/scala/unit/kafka/utils/ShutdownableThreadTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ShutdownableThreadTest.scala @@ -20,7 +20,7 @@ import java.util.concurrent.{CountDownLatch, TimeUnit} import org.apache.kafka.common.internals.FatalExitError import org.apache.kafka.server.util.ShutdownableThread import org.junit.jupiter.api.{AfterEach, Test} -import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} class ShutdownableThreadTest { @@ -51,4 +51,19 @@ class ShutdownableThreadTest { assertEquals(1, statusCodeOption.get) } + @Test + def testIsThreadStarted(): Unit = { + val latch = new CountDownLatch(1) + val thread = new ShutdownableThread("shutdownable-thread-test") { + override def doWork(): Unit = { + latch.countDown() + } + } + assertFalse(thread.isStarted) + thread.start() + latch.await() + assertTrue(thread.isStarted) + + thread.shutdown() + } } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 4bf6a82854..61cd6c4b01 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -133,12 +133,7 @@ object TestUtils extends Logging { /** * Create a temporary relative directory */ - def tempRelativeDir(parent: String): File = { - val parentFile = new File(parent) - parentFile.mkdirs() - - JTestUtils.tempDirectory(parentFile.toPath, null) - } + def tempRelativeDir(root: String): File = JTestUtils.tempRelativeDir(root) /** * Create a random log directory in the format - used for Kafka partition logs. @@ -373,7 +368,7 @@ object TestUtils extends Logging { props.put(KafkaConfig.DeleteTopicEnableProp, enableDeleteTopic.toString) props.put(KafkaConfig.LogDeleteDelayMsProp, "1000") props.put(KafkaConfig.ControlledShutdownRetryBackoffMsProp, "100") - props.put(KafkaConfig.LogCleanerDedupeBufferSizeProp, "2097152") + props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152") props.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") if (!props.containsKey(KafkaConfig.OffsetsTopicPartitionsProp)) props.put(KafkaConfig.OffsetsTopicPartitionsProp, "5") diff --git a/docker/README.md b/docker/README.md index 0232604e61..54d15c04fe 100644 --- a/docker/README.md +++ b/docker/README.md @@ -63,6 +63,26 @@ rc_docker_image: apache/kafka:3.6.0-rc0 promoted_docker_image: apache/kafka:3.6.0 ``` +Cron job for checking CVEs in supported docker images +----------------------------------------------------- + +- `Docker Image CVE Scanner` Github Action Workflow (present in `.github/workflows/docker_scan.yml`) will run nightly CVE scans and generate reports for docker image tags mentioned in the `supported_image_tag` array. +- This workflow is branch independent. Only the workflow in trunk, i.e. the default branch will be considered. +- In case a Critical or High CVE is detected, the workflow will fail. +- It will generate the scan reports that can be checked by the community. +- For every new release, this should be updated with the latest supported docker images. +- For example:- +``` +For supporting apache/kafka:3.6.0, apache/kafka:latest and apache/kafka:3.7.0-rc0, supported_image_tag array should be +supported_image_tag: ['3.6.0', 'latest', '3.7.0-rc0'] +``` +- When RC for a version gets changed or when a bug fix release happens, this should be updated as well. +- For example:- +``` +For supporting apache/kafka:3.6.1, apache/kafka:latest and apache/kafka:3.7.0-rc1, tag array should be +supported_image_tag: ['3.6.1', 'latest', '3.7.0-rc1'] +``` + Local Setup ----------- diff --git a/docs/connect.html b/docs/connect.html index 954c7c647d..c06879813f 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -162,19 +162,22 @@
    predicate to selectively filter certain messages. -
  • InsertHeader - Add a header using static data
  • -
  • HeadersFrom - Copy or move fields in the key or value to the record headers
  • -
  • DropHeaders - Remove headers by name
  • +
  • Cast - Cast fields or the entire key or value to a specific type
  • +
  • DropHeaders - Remove headers by name
  • +
  • ExtractField - Extract a specific field from Struct and Map and include only this field in results
  • +
  • Filter - Removes messages from all further processing. This is used with a predicate to selectively filter certain messages
  • +
  • Flatten - Flatten a nested data structure
  • +
  • HeaderFrom - Copy or move fields in the key or value to the record headers
  • +
  • HoistField - Wrap the entire event as a single field inside a Struct or a Map
  • +
  • InsertField - Add a field using either static data or record metadata
  • +
  • InsertHeader - Add a header using static data
  • +
  • MaskField - Replace field with valid null value for the type (0, empty string, etc) or custom replacement (non-empty string or numeric value only)
  • +
  • RegexRouter - modify the topic of a record based on original topic, replacement string and a regular expression
  • +
  • ReplaceField - Filter or rename fields
  • +
  • SetSchemaMetadata - modify the schema name or version
  • +
  • TimestampConverter - Convert timestamps between different formats
  • +
  • TimestampRouter - Modify the topic of a record based on original topic and timestamp. Useful when using a sink that needs to write to different tables or indexes based on timestamps
  • +
  • ValueToKey - Replace the record key with a new key formed from a subset of fields in the record value
  • Details on how to configure each transformation are listed below:

    diff --git a/docs/ops.html b/docs/ops.html index 92f6761555..ce1e0b4109 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -2537,11 +2537,26 @@

    < The total amount of buffer memory that is not being used (either unallocated or in the free list). kafka.producer:type=producer-metrics,client-id=([-.\w]+) + + buffer-exhausted-rate + The average per-second number of record sends that are dropped due to buffer exhaustion + kafka.producer:type=producer-metrics,client-id=([-.\w]+) + + + buffer-exhausted-total + The total number of record sends that are dropped due to buffer exhaustion + kafka.producer:type=producer-metrics,client-id=([-.\w]+) + bufferpool-wait-time The fraction of time an appender waits for space allocation. kafka.producer:type=producer-metrics,client-id=([-.\w]+) + + bufferpool-wait-ratio + The fraction of time an appender waits for space allocation. + kafka.producer:type=producer-metrics,client-id=([-.\w]+) + bufferpool-wait-time-total *Deprecated* The total time an appender waits for space allocation in nanoseconds. Replacement is bufferpool-wait-time-ns-total @@ -2582,7 +2597,11 @@

    < The total time the Producer spent aborting transactions in nanoseconds (for EOS). kafka.producer:type=producer-metrics,client-id=([-.\w]+) - + + metadata-wait-time-ns-total + the total time in nanoseconds that has spent waiting for metadata from the Kafka broker + kafka.producer:type=producer-metrics,client-id=([-.\w]+) +

    Producer Sender Metrics
    @@ -3780,12 +3799,6 @@

    ZooKeeper to KRaft Migration

    -

    - ZooKeeper to KRaft migration is considered an Early Access feature and is not recommended for production clusters. - Please report issues with ZooKeeper to KRaft migration using the - project JIRA and the "kraft" component. -

    -

    Terminology

    • Brokers that are in ZK mode store their metadata in Apache ZooKepeer. This is the old mode of handling metadata.
    • @@ -3807,7 +3820,7 @@

      Migration Phases

      Limitations

      • While a cluster is being migrated from ZK mode to KRaft mode, we do not support changing the metadata - version (also known as the inter.broker.protocol version.) Please do not attempt to do this during + version (also known as the inter.broker.protocol.version.) Please do not attempt to do this during a migration, or you may break the cluster.
      • After the migration has been finalized, it is not possible to revert back to ZooKeeper mode.
      • As noted above, some features are not fully implemented in KRaft mode. If you are diff --git a/docs/streams/developer-guide/config-streams.html b/docs/streams/developer-guide/config-streams.html index 288946d33a..40b3594880 100644 --- a/docs/streams/developer-guide/config-streams.html +++ b/docs/streams/developer-guide/config-streams.html @@ -97,6 +97,7 @@
      • diff --git a/docs/streams/developer-guide/processor-api.html b/docs/streams/developer-guide/processor-api.html index ccb03ce7b5..e25ecd601a 100644 --- a/docs/streams/developer-guide/processor-api.html +++ b/docs/streams/developer-guide/processor-api.html @@ -51,6 +51,7 @@
      • Enable or Disable Fault Tolerance of State Stores (Store Changelogs)
      • Timestamped State Stores
      • Versioned Key-Value State Stores
      • +
      • Readonly State Stores
      • Implementing Custom State Stores
      @@ -466,6 +467,18 @@

      stores to rebuild state from changelog.

    +
    +

    ReadOnly State Stores

    +

    A read-only state store materialized the data from its input topic. It also uses the input topic + for fault-tolerance, and thus does not have an additional changelog topic (the input topic is + re-used as changelog). Thus, the input topic should be configured with log compaction. + Note that no other processor should modify the content of the state store, and the only writer + should be the associated "state update processor"; other processors may read the content of the + read-only store.

    + +

    note: beware of the partitioning requirements when using read-only state stores for lookups during + processing. You might want to make sure the original changelog topic is co-partitioned with the processors + reading the read-only statestore.

    Implementing Custom State Stores

    diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index 1454407740..0f819cb384 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -133,6 +133,14 @@

    < More details about the new config StreamsConfig#TOPOLOGY_OPTIMIZATION can be found in KIP-295.

    +

    Streams API changes in 3.8.0

    +

    + The Processor API now support so-called read-only state stores, added via + KIP-813. + These stores don't have a dedicated changelog topic, but use their source topic for fault-tolerance, + simlar to KTables with source-topic optimization enabled. +

    +

    Streams API changes in 3.7.0

    We added a new method to KafkaStreams, namely KafkaStreams#setStandbyUpdateListener() in diff --git a/docs/upgrade.html b/docs/upgrade.html index 33009f062e..6488ead802 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -106,9 +106,9 @@

    Notable changes in 3 -

    Upgrading to 3.6.1 from any version 0.8.x through 3.5.x

    +

    Upgrading to 3.6.2 from any version 0.8.x through 3.5.x

    -
    Upgrading ZooKeeper-based clusters
    +
    Upgrading ZooKeeper-based clusters

    If you are upgrading from a version prior to 2.1.x, please see the note in step 5 below about the change to the schema used to store consumer offsets. Once you have changed the inter.broker.protocol.version to the latest version, it will not be possible to downgrade to a version prior to 2.1.

    @@ -149,7 +149,7 @@
    Upgrading ZooKeeper-based clus -
    Upgrading KRaft-based clusters
    +
    Upgrading KRaft-based clusters

    If you are upgrading from a version prior to 3.3.0, please see the note in step 3 below. Once you have changed the metadata.version to the latest version, it will not be possible to downgrade to a version prior to 3.3-IV0.

    For a rolling upgrade:

    diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index f3af21a0dd..722b2178a7 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -110,7 +110,7 @@ versions += [ javassist: "3.29.2-GA", jetty: "9.4.53.v20231009", jersey: "2.39.1", - jline: "3.22.0", + jline: "3.25.1", jmh: "1.37", hamcrest: "2.2", scalaLogging: "3.9.4", @@ -121,7 +121,7 @@ versions += [ jopt: "5.0.4", jose4j: "0.9.4", junit: "5.10.2", - jqwik: "1.7.4", + jqwik: "1.8.3", kafka_0100: "0.10.0.1", kafka_0101: "0.10.1.1", kafka_0102: "0.10.2.2", @@ -162,13 +162,15 @@ versions += [ // https://github.com/scalameta/scalafmt/releases/tag/v3.1.0. scalafmt: "3.7.14", scalaJava8Compat : "1.0.2", - scoverage: "1.9.3", - slf4j: "2.0.9", + scoverage: "2.0.11", + slf4j: "1.7.36", snappy: "1.1.10.5", spotbugs: "4.8.0", zinc: "1.9.2", - zookeeper: "3.8.3", + zookeeper: "3.8.4", zstd: "1.5.5-11", + + // AutoMQ inject start commonLang: "3.12.0", commonio: "2.15.1", commonMath3: "3.6.1", @@ -179,6 +181,7 @@ versions += [ bucket4j:"8.5.0", jna:"5.2.0", guava:"32.0.1-jre", + // AutoMQ inject end ] libs += [ diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java index a2363b4822..fb4dc782b6 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java @@ -80,7 +80,6 @@ import java.util.Map; import java.util.OptionalInt; import java.util.Properties; -import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; @@ -498,29 +497,24 @@ public CompletableFuture listGroups( ); } - final Set existingPartitionSet = runtime.partitions(); - - if (existingPartitionSet.isEmpty()) { - return CompletableFuture.completedFuture(new ListGroupsResponseData()); - } - - final List>> futures = - new ArrayList<>(); - - for (TopicPartition tp : existingPartitionSet) { - futures.add(runtime.scheduleReadOperation( + final List>> futures = FutureUtils.mapExceptionally( + runtime.scheduleReadAllOperation( "list-groups", - tp, - (coordinator, lastCommittedOffset) -> coordinator.listGroups(request.statesFilter(), request.typesFilter(), lastCommittedOffset) - ).exceptionally(exception -> { + (coordinator, lastCommittedOffset) -> coordinator.listGroups( + request.statesFilter(), + request.typesFilter(), + lastCommittedOffset + ) + ), + exception -> { exception = Errors.maybeUnwrapException(exception); if (exception instanceof NotCoordinatorException) { return Collections.emptyList(); } else { throw new CompletionException(exception); } - })); - } + } + ); return FutureUtils .combineFutures(futures, ArrayList::new, List::addAll) @@ -864,7 +858,8 @@ public CompletableFuture commitTransactionalOffsets request.producerId(), request.producerEpoch(), Duration.ofMillis(config.offsetCommitTimeoutMs), - coordinator -> coordinator.commitTransactionalOffset(context, request) + coordinator -> coordinator.commitTransactionalOffset(context, request), + context.apiVersion() ).exceptionally(exception -> handleOperationException( "txn-commit-offset", request, @@ -963,23 +958,21 @@ public void onPartitionsDeleted( ) throws ExecutionException, InterruptedException { throwIfNotActive(); - final Set existingPartitionSet = runtime.partitions(); - final List> futures = new ArrayList<>(existingPartitionSet.size()); - - existingPartitionSet.forEach(partition -> futures.add( - runtime.scheduleWriteOperation( - "on-partition-deleted", - partition, - Duration.ofMillis(config.offsetCommitTimeoutMs), - coordinator -> coordinator.onPartitionsDeleted(topicPartitions) - ).exceptionally(exception -> { - log.error("Could not delete offsets for deleted partitions {} in coordinator {} due to: {}.", - partition, partition, exception.getMessage(), exception); - return null; - }) - )); - - CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get(); + CompletableFuture.allOf( + FutureUtils.mapExceptionally( + runtime.scheduleWriteAllOperation( + "on-partition-deleted", + Duration.ofMillis(config.offsetCommitTimeoutMs), + coordinator -> coordinator.onPartitionsDeleted(topicPartitions) + ), + exception -> { + log.error("Could not delete offsets for deleted partitions {} due to: {}.", + topicPartitions, exception.getMessage(), exception + ); + return null; + } + ).toArray(new CompletableFuture[0]) + ).get(); } /** @@ -1103,6 +1096,14 @@ private OUT handleOperationException( operationName, operationInput, exception.getMessage(), exception); return handler.apply(Errors.UNKNOWN_SERVER_ERROR, null); + case NETWORK_EXCEPTION: + // When committing offsets transactionally, we now verify the transaction with the + // transaction coordinator. Verification can fail with `NETWORK_EXCEPTION`, a + // retriable error which older clients may not expect and retry correctly. We + // translate the error to `COORDINATOR_LOAD_IN_PROGRESS` because it causes clients + // to retry the request without an unnecessary coordinator lookup. + return handler.apply(Errors.COORDINATOR_LOAD_IN_PROGRESS, null); + case UNKNOWN_TOPIC_OR_PARTITION: case NOT_ENOUGH_REPLICAS: case REQUEST_TIMED_OUT: diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java index b55dd91cc1..12c194c331 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java @@ -388,7 +388,7 @@ public CoordinatorResult describeGroups( } /** - * Gets or maybe creates a consumer group. + * Gets or maybe creates a consumer group without updating the groups map. + * The group will be materialized during the replay. * * @param groupId The group id. * @param createIfNotExists A boolean indicating whether the group should be - * created if it does not exist. + * created if it does not exist or is an empty classic group. + * @param records The record list to which the group tombstones are written + * if the group is empty and is a classic group. * * @return A ConsumerGroup. * @throws GroupIdNotFoundException if the group does not exist and createIfNotExists is false or @@ -593,7 +600,8 @@ public List describeGroups( */ ConsumerGroup getOrMaybeCreateConsumerGroup( String groupId, - boolean createIfNotExists + boolean createIfNotExists, + List records ) throws GroupIdNotFoundException { Group group = groups.get(groupId); @@ -601,6 +609,76 @@ ConsumerGroup getOrMaybeCreateConsumerGroup( throw new GroupIdNotFoundException(String.format("Consumer group %s not found.", groupId)); } + if (group == null || (createIfNotExists && maybeDeleteEmptyClassicGroup(group, records))) { + return new ConsumerGroup(snapshotRegistry, groupId, metrics); + } else { + if (group.type() == CONSUMER) { + return (ConsumerGroup) group; + } else { + // We don't support upgrading/downgrading between protocols at the moment so + // we throw an exception if a group exists with the wrong type. + throw new GroupIdNotFoundException(String.format("Group %s is not a consumer group.", groupId)); + } + } + } + + /** + * Gets a consumer group by committed offset. + * + * @param groupId The group id. + * @param committedOffset A specified committed offset corresponding to this shard. + * + * @return A ConsumerGroup. + * @throws GroupIdNotFoundException if the group does not exist or is not a consumer group. + */ + public ConsumerGroup consumerGroup( + String groupId, + long committedOffset + ) throws GroupIdNotFoundException { + Group group = group(groupId, committedOffset); + + if (group.type() == CONSUMER) { + return (ConsumerGroup) group; + } else { + // We don't support upgrading/downgrading between protocols at the moment so + // we throw an exception if a group exists with the wrong type. + throw new GroupIdNotFoundException(String.format("Group %s is not a consumer group.", + groupId)); + } + } + + /** + * An overloaded method of {@link GroupMetadataManager#consumerGroup(String, long)} + */ + ConsumerGroup consumerGroup( + String groupId + ) throws GroupIdNotFoundException { + return consumerGroup(groupId, Long.MAX_VALUE); + } + + /** + * The method should be called on the replay path. + * Gets or maybe creates a consumer group and updates the groups map if a new group is created. + * + * @param groupId The group id. + * @param createIfNotExists A boolean indicating whether the group should be + * created if it does not exist. + * + * @return A ConsumerGroup. + * @throws IllegalStateException if the group does not exist and createIfNotExists is false or + * if the group is not a consumer group. + * Package private for testing. + */ + ConsumerGroup getOrMaybeCreatePersistedConsumerGroup( + String groupId, + boolean createIfNotExists + ) throws GroupIdNotFoundException { + Group group = groups.get(groupId); + + if (group == null && !createIfNotExists) { + throw new IllegalStateException(String.format("Consumer group %s not found.", groupId)); + } + if (group == null) { ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId, metrics); groups.put(groupId, consumerGroup); @@ -612,7 +690,7 @@ ConsumerGroup getOrMaybeCreateConsumerGroup( } else { // We don't support upgrading/downgrading between protocols at the moment so // we throw an exception if a group exists with the wrong type. - throw new GroupIdNotFoundException(String.format("Group %s is not a consumer group.", groupId)); + throw new IllegalStateException(String.format("Group %s is not a consumer group.", groupId)); } } } @@ -682,31 +760,6 @@ public ClassicGroup classicGroup( } } - /** - * Gets a consumer group by committed offset. - * - * @param groupId The group id. - * @param committedOffset A specified committed offset corresponding to this shard. - * - * @return A ConsumerGroup. - * @throws GroupIdNotFoundException if the group does not exist or is not a consumer group. - */ - public ConsumerGroup consumerGroup( - String groupId, - long committedOffset - ) throws GroupIdNotFoundException { - Group group = group(groupId, committedOffset); - - if (group.type() == CONSUMER) { - return (ConsumerGroup) group; - } else { - // We don't support upgrading/downgrading between protocols at the moment so - // we throw an exception if a group exists with the wrong type. - throw new GroupIdNotFoundException(String.format("Group %s is not a consumer group.", - groupId)); - } - } - /** * Removes the group. * @@ -1015,7 +1068,7 @@ private CoordinatorResult consumerGr // Get or create the consumer group. boolean createIfNotExists = memberEpoch == 0; - final ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, createIfNotExists); + final ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, createIfNotExists, records); throwIfConsumerGroupIsFull(group, memberId); // Get or create the member. @@ -1058,7 +1111,6 @@ private CoordinatorResult consumerGr } } - int groupEpoch = group.groupEpoch(); Map subscriptionMetadata = group.subscriptionMetadata(); @@ -1166,38 +1218,17 @@ private CoordinatorResult consumerGr } } - // 3. Reconcile the member's assignment with the target assignment. This is only required if - // the member is not stable or if a new target assignment has been installed. - boolean assignmentUpdated = false; - if (updatedMember.state() != ConsumerGroupMember.MemberState.STABLE || updatedMember.targetMemberEpoch() != targetAssignmentEpoch) { - ConsumerGroupMember prevMember = updatedMember; - updatedMember = new CurrentAssignmentBuilder(updatedMember) - .withTargetAssignment(targetAssignmentEpoch, targetAssignment) - .withCurrentPartitionEpoch(group::currentPartitionEpoch) - .withOwnedTopicPartitions(ownedTopicPartitions) - .build(); - - // Checking the reference is enough here because a new instance - // is created only when the state has changed. - if (updatedMember != prevMember) { - assignmentUpdated = true; - records.add(newCurrentAssignmentRecord(groupId, updatedMember)); - - log.info("[GroupId {}] Member {} transitioned from {} to {}.", - groupId, memberId, member.currentAssignmentSummary(), updatedMember.currentAssignmentSummary()); - - if (updatedMember.state() == ConsumerGroupMember.MemberState.REVOKING) { - scheduleConsumerGroupRevocationTimeout( - groupId, - memberId, - updatedMember.rebalanceTimeoutMs(), - updatedMember.memberEpoch() - ); - } else { - cancelConsumerGroupRevocationTimeout(groupId, memberId); - } - } - } + // 3. Reconcile the member's assignment with the target assignment if the member is not + // fully reconciled yet. + updatedMember = maybeReconcile( + groupId, + updatedMember, + group::currentPartitionEpoch, + targetAssignmentEpoch, + targetAssignment, + ownedTopicPartitions, + records + ); scheduleConsumerGroupSessionTimeout(groupId, memberId); @@ -1208,16 +1239,77 @@ private CoordinatorResult consumerGr .setHeartbeatIntervalMs(consumerGroupHeartbeatIntervalMs); // The assignment is only provided in the following cases: - // 1. The member reported its owned partitions; - // 2. The member just joined or rejoined to group (epoch equals to zero); - // 3. The member's assignment has been updated. - if (ownedTopicPartitions != null || memberEpoch == 0 || assignmentUpdated) { + // 1. The member sent a full request. It does so when joining or rejoining the group with zero + // as the member epoch; or on any errors (e.g. timeout). We use all the non-optional fields + // (rebalanceTimeoutMs, subscribedTopicNames and ownedTopicPartitions) to detect a full request + // as those must be set in a full request. + // 2. The member's assignment has been updated. + boolean isFullRequest = memberEpoch == 0 || (rebalanceTimeoutMs != -1 && subscribedTopicNames != null && ownedTopicPartitions != null); + if (isFullRequest || hasAssignedPartitionsChanged(member, updatedMember)) { response.setAssignment(createResponseAssignment(updatedMember)); } return new CoordinatorResult<>(records, response); } + /** + * Reconciles the current assignment of the member towards the target assignment if needed. + * + * @param groupId The group id. + * @param member The member to reconcile. + * @param currentPartitionEpoch The function returning the current epoch of + * a given partition. + * @param targetAssignmentEpoch The target assignment epoch. + * @param targetAssignment The target assignment. + * @param ownedTopicPartitions The list of partitions owned by the member. This + * is reported in the ConsumerGroupHeartbeat API and + * it could be null if not provided. + * @param records The list to accumulate any new records. + * @return The received member if no changes have been made; or a new + * member containing the new assignment. + */ + private ConsumerGroupMember maybeReconcile( + String groupId, + ConsumerGroupMember member, + BiFunction currentPartitionEpoch, + int targetAssignmentEpoch, + Assignment targetAssignment, + List ownedTopicPartitions, + List records + ) { + if (member.isReconciledTo(targetAssignmentEpoch)) { + return member; + } + + ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(targetAssignmentEpoch, targetAssignment) + .withCurrentPartitionEpoch(currentPartitionEpoch) + .withOwnedTopicPartitions(ownedTopicPartitions) + .build(); + + if (!updatedMember.equals(member)) { + records.add(newCurrentAssignmentRecord(groupId, updatedMember)); + + log.info("[GroupId {}] Member {} new assignment state: epoch={}, previousEpoch={}, state={}, " + + "assignedPartitions={} and revokedPartitions={}.", + groupId, updatedMember.memberId(), updatedMember.memberEpoch(), updatedMember.previousMemberEpoch(), updatedMember.state(), + assignmentToString(updatedMember.assignedPartitions()), assignmentToString(updatedMember.partitionsPendingRevocation())); + + if (updatedMember.state() == MemberState.UNREVOKED_PARTITIONS) { + scheduleConsumerGroupRebalanceTimeout( + groupId, + updatedMember.memberId(), + updatedMember.memberEpoch(), + updatedMember.rebalanceTimeoutMs() + ); + } else { + cancelConsumerGroupRebalanceTimeout(groupId, updatedMember.memberId()); + } + } + + return updatedMember; + } + private void removeMemberAndCancelTimers( List records, String groupId, @@ -1244,7 +1336,7 @@ private CoordinatorResult consumerGr String memberId, int memberEpoch ) throws ApiException { - ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroup group = consumerGroup(groupId); List records; if (instanceId == null) { ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false); @@ -1353,7 +1445,7 @@ private void removeMember(List records, String groupId, String memberId) */ private void cancelTimers(String groupId, String memberId) { cancelConsumerGroupSessionTimeout(groupId, memberId); - cancelConsumerGroupRevocationTimeout(groupId, memberId); + cancelConsumerGroupRebalanceTimeout(groupId, memberId); } /** @@ -1369,7 +1461,7 @@ private void scheduleConsumerGroupSessionTimeout( String key = consumerGroupSessionTimeoutKey(groupId, memberId); timer.schedule(key, consumerGroupSessionTimeoutMs, TimeUnit.MILLISECONDS, true, () -> { try { - ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroup group = consumerGroup(groupId); ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false); log.info("[GroupId {}] Member {} fenced from the group because its session expired.", groupId, memberId); @@ -1400,35 +1492,35 @@ private void cancelConsumerGroupSessionTimeout( } /** - * Schedules a revocation timeout for the member. + * Schedules a rebalance timeout for the member. * * @param groupId The group id. * @param memberId The member id. - * @param revocationTimeoutMs The revocation timeout. - * @param expectedMemberEpoch The expected member epoch. + * @param memberEpoch The member epoch. + * @param rebalanceTimeoutMs The rebalance timeout. */ - private void scheduleConsumerGroupRevocationTimeout( + private void scheduleConsumerGroupRebalanceTimeout( String groupId, String memberId, - long revocationTimeoutMs, - int expectedMemberEpoch + int memberEpoch, + int rebalanceTimeoutMs ) { - String key = consumerGroupRevocationTimeoutKey(groupId, memberId); - timer.schedule(key, revocationTimeoutMs, TimeUnit.MILLISECONDS, true, () -> { + String key = consumerGroupRebalanceTimeoutKey(groupId, memberId); + timer.schedule(key, rebalanceTimeoutMs, TimeUnit.MILLISECONDS, true, () -> { try { - ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroup group = consumerGroup(groupId); ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false); - if (member.state() != ConsumerGroupMember.MemberState.REVOKING || - member.memberEpoch() != expectedMemberEpoch) { - log.debug("[GroupId {}] Ignoring revocation timeout for {} because the member " + - "state does not match the expected state.", groupId, memberId); + if (member.memberEpoch() == memberEpoch) { + log.info("[GroupId {}] Member {} fenced from the group because " + + "it failed to transition from epoch {} within {}ms.", + groupId, memberId, memberEpoch, rebalanceTimeoutMs); + return new CoordinatorResult<>(consumerGroupFenceMember(group, member)); + } else { + log.debug("[GroupId {}] Ignoring rebalance timeout for {} because the member " + + "left the epoch {}.", groupId, memberId, memberEpoch); return new CoordinatorResult<>(Collections.emptyList()); } - - log.info("[GroupId {}] Member {} fenced from the group because " + - "it failed to revoke partitions within {}ms.", groupId, memberId, revocationTimeoutMs); - return new CoordinatorResult<>(consumerGroupFenceMember(group, member)); } catch (GroupIdNotFoundException ex) { log.debug("[GroupId {}] Could not fence {}} because the group does not exist.", groupId, memberId); @@ -1442,16 +1534,16 @@ private void scheduleConsumerGroupRevocationTimeout( } /** - * Cancels the revocation timeout of the member. + * Cancels the rebalance timeout of the member. * * @param groupId The group id. * @param memberId The member id. */ - private void cancelConsumerGroupRevocationTimeout( + private void cancelConsumerGroupRebalanceTimeout( String groupId, String memberId ) { - timer.cancel(consumerGroupRevocationTimeoutKey(groupId, memberId)); + timer.cancel(consumerGroupRebalanceTimeoutKey(groupId, memberId)); } /** @@ -1511,7 +1603,7 @@ public void replay( String groupId = key.groupId(); String memberId = key.memberId(); - ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, value != null); + ConsumerGroup consumerGroup = getOrMaybeCreatePersistedConsumerGroup(groupId, value != null); Set oldSubscribedTopicNames = new HashSet<>(consumerGroup.subscribedTopicNames()); if (value != null) { @@ -1623,10 +1715,10 @@ public void replay( String groupId = key.groupId(); if (value != null) { - ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, true); + ConsumerGroup consumerGroup = getOrMaybeCreatePersistedConsumerGroup(groupId, true); consumerGroup.setGroupEpoch(value.epoch()); } else { - ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroup consumerGroup = getOrMaybeCreatePersistedConsumerGroup(groupId, false); if (!consumerGroup.members().isEmpty()) { throw new IllegalStateException("Received a tombstone record to delete group " + groupId + " but the group still has " + consumerGroup.members().size() + " members."); @@ -1658,7 +1750,7 @@ public void replay( ConsumerGroupPartitionMetadataValue value ) { String groupId = key.groupId(); - ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroup consumerGroup = getOrMaybeCreatePersistedConsumerGroup(groupId, false); if (value != null) { Map subscriptionMetadata = new HashMap<>(); @@ -1684,7 +1776,7 @@ public void replay( ) { String groupId = key.groupId(); String memberId = key.memberId(); - ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroup consumerGroup = getOrMaybeCreatePersistedConsumerGroup(groupId, false); if (value != null) { consumerGroup.updateTargetAssignment(memberId, Assignment.fromRecord(value)); @@ -1706,7 +1798,7 @@ public void replay( ConsumerGroupTargetAssignmentMetadataValue value ) { String groupId = key.groupId(); - ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroup consumerGroup = getOrMaybeCreatePersistedConsumerGroup(groupId, false); if (value != null) { consumerGroup.setTargetAssignmentEpoch(value.assignmentEpoch()); @@ -1732,7 +1824,7 @@ public void replay( ) { String groupId = key.groupId(); String memberId = key.memberId(); - ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroup consumerGroup = getOrMaybeCreatePersistedConsumerGroup(groupId, false); ConsumerGroupMember oldMember = consumerGroup.getOrMaybeCreateMember(memberId, false); if (value != null) { @@ -1744,10 +1836,8 @@ public void replay( ConsumerGroupMember newMember = new ConsumerGroupMember.Builder(oldMember) .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) .setPreviousMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) - .setTargetMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) .setAssignedPartitions(Collections.emptyMap()) .setPartitionsPendingRevocation(Collections.emptyMap()) - .setPartitionsPendingAssignment(Collections.emptyMap()) .build(); consumerGroup.updateMember(newMember); } @@ -1796,12 +1886,12 @@ public void onLoaded() { consumerGroup.members().forEach((memberId, member) -> { log.debug("Loaded member {} in consumer group {}.", memberId, groupId); scheduleConsumerGroupSessionTimeout(groupId, memberId); - if (member.state() == ConsumerGroupMember.MemberState.REVOKING) { - scheduleConsumerGroupRevocationTimeout( + if (member.state() == MemberState.UNREVOKED_PARTITIONS) { + scheduleConsumerGroupRebalanceTimeout( groupId, - memberId, - member.rebalanceTimeoutMs(), - member.memberEpoch() + member.memberId(), + member.memberEpoch(), + member.rebalanceTimeoutMs() ); } }); @@ -1834,8 +1924,8 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe return "session-timeout-" + groupId + "-" + memberId; } - public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) { - return "revocation-timeout-" + groupId + "-" + memberId; + public static String consumerGroupRebalanceTimeoutKey(String groupId, String memberId) { + return "rebalance-timeout-" + groupId + "-" + memberId; } /** @@ -1922,6 +2012,7 @@ public CoordinatorResult classicGroupJoin( CompletableFuture responseFuture ) { CoordinatorResult result = EMPTY_RESULT; + List records = new ArrayList<>(); String groupId = request.groupId(); String memberId = request.memberId(); @@ -1939,6 +2030,7 @@ public CoordinatorResult classicGroupJoin( // Group is created if it does not exist and the member id is UNKNOWN. if member // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND ClassicGroup group; + maybeDeleteEmptyConsumerGroup(groupId, records); boolean isNewGroup = !groups.containsKey(groupId); try { group = getOrMaybeCreateClassicGroup(groupId, isUnknownMember); @@ -1987,7 +2079,7 @@ public CoordinatorResult classicGroupJoin( } }); - List records = Collections.singletonList( + records.add( RecordHelpers.newEmptyGroupMetadataRecord(group, metadataImage.features().metadataVersion()) ); @@ -3469,12 +3561,25 @@ private void removeCurrentMemberFromClassicGroup( * @param groupId The id of the group to be deleted. It has been checked in {@link GroupMetadataManager#validateDeleteGroup}. * @param records The record list to populate. */ - public void deleteGroup( + public void createGroupTombstoneRecords( String groupId, List records ) { // At this point, we have already validated the group id, so we know that the group exists and that no exception will be thrown. - group(groupId).createGroupTombstoneRecords(records); + createGroupTombstoneRecords(group(groupId), records); + } + + /** + * Populates the record list passed in with record to update the state machine. + * + * @param group The group to be deleted. + * @param records The record list to populate. + */ + public void createGroupTombstoneRecords( + Group group, + List records + ) { + group.createGroupTombstoneRecords(records); } /** @@ -3496,7 +3601,55 @@ void validateDeleteGroup(String groupId) throws ApiException { public void maybeDeleteGroup(String groupId, List records) { Group group = groups.get(groupId); if (group != null && group.isEmpty()) { - deleteGroup(groupId, records); + createGroupTombstoneRecords(groupId, records); + } + } + + /** + * @return true if the group is an empty classic group. + */ + private static boolean isEmptyClassicGroup(Group group) { + return group != null && group.type() == CLASSIC && group.isEmpty(); + } + + /** + * @return true if the group is an empty consumer group. + */ + private static boolean isEmptyConsumerGroup(Group group) { + return group != null && group.type() == CONSUMER && group.isEmpty(); + } + + /** + * Write tombstones for the group if it's empty and is a classic group. + * + * @param group The group to be deleted. + * @param records The list of records to delete the group. + * + * @return true if the group is empty + */ + private boolean maybeDeleteEmptyClassicGroup(Group group, List records) { + if (isEmptyClassicGroup(group)) { + // Delete the classic group by adding tombstones. + // There's no need to remove the group as the replay of tombstones removes it. + if (group != null) createGroupTombstoneRecords(group, records); + return true; + } + return false; + } + + /** + * Delete and write tombstones for the group if it's empty and is a consumer group. + * + * @param groupId The group id to be deleted. + * @param records The list of records to delete the group. + */ + private void maybeDeleteEmptyConsumerGroup(String groupId, List records) { + Group group = groups.get(groupId, Long.MAX_VALUE); + if (isEmptyConsumerGroup(group)) { + // Add tombstones for the previous consumer group. The tombstones won't actually be + // replayed because its coordinator result has a non-null appendFuture. + createGroupTombstoneRecords(group, records); + removeGroup(groupId); } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java index 47ba36c84a..c2bd093d6a 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java @@ -82,16 +82,7 @@ public static Record newMemberSubscriptionRecord( .setSubscribedTopicNames(member.subscribedTopicNames()) .setSubscribedTopicRegex(member.subscribedTopicRegex()) .setServerAssignor(member.serverAssignorName().orElse(null)) - .setRebalanceTimeoutMs(member.rebalanceTimeoutMs()) - .setAssignors(member.clientAssignors().stream().map(assignorState -> - new ConsumerGroupMemberMetadataValue.Assignor() - .setName(assignorState.name()) - .setReason(assignorState.reason()) - .setMinimumVersion(assignorState.minimumVersion()) - .setMaximumVersion(assignorState.maximumVersion()) - .setVersion(assignorState.metadata().version()) - .setMetadata(assignorState.metadata().metadata().array()) - ).collect(Collectors.toList())), + .setRebalanceTimeoutMs(member.rebalanceTimeoutMs()), (short) 0 ) ); @@ -346,10 +337,9 @@ public static Record newCurrentAssignmentRecord( new ConsumerGroupCurrentMemberAssignmentValue() .setMemberEpoch(member.memberEpoch()) .setPreviousMemberEpoch(member.previousMemberEpoch()) - .setTargetMemberEpoch(member.targetMemberEpoch()) + .setState(member.state().value()) .setAssignedPartitions(toTopicPartitions(member.assignedPartitions())) - .setPartitionsPendingRevocation(toTopicPartitions(member.partitionsPendingRevocation())) - .setPartitionsPendingAssignment(toTopicPartitions(member.partitionsPendingAssignment())), + .setPartitionsPendingRevocation(toTopicPartitions(member.partitionsPendingRevocation())), (short) 0 ) ); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java index 7fe92c5bd8..cbbe45cc61 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java @@ -16,8 +16,13 @@ */ package org.apache.kafka.coordinator.group; +import org.apache.kafka.common.Uuid; + +import java.util.Iterator; +import java.util.Map; import java.util.OptionalInt; import java.util.OptionalLong; +import java.util.Set; public class Utils { private Utils() {} @@ -37,4 +42,31 @@ public static OptionalInt ofSentinel(int value) { public static OptionalLong ofSentinel(long value) { return value != -1 ? OptionalLong.of(value) : OptionalLong.empty(); } + + /** + * @return The provided assignment as a String. + * + * Example: + * [topicid1-0, topicid1-1, topicid2-0, topicid2-1] + */ + public static String assignmentToString( + Map> assignment + ) { + StringBuilder builder = new StringBuilder("["); + Iterator>> topicsIterator = assignment.entrySet().iterator(); + while (topicsIterator.hasNext()) { + Map.Entry> entry = topicsIterator.next(); + Iterator partitionsIterator = entry.getValue().iterator(); + while (partitionsIterator.hasNext()) { + builder.append(entry.getKey()); + builder.append("-"); + builder.append(partitionsIterator.next()); + if (partitionsIterator.hasNext() || topicsIterator.hasNext()) { + builder.append(", "); + } + } + } + builder.append("]"); + return builder.toString(); + } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/Assignment.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/Assignment.java index 9cf6b521dd..f70601ec0c 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/Assignment.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/Assignment.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashSet; import java.util.Map; @@ -31,52 +30,17 @@ * An immutable assignment for a member. */ public class Assignment { - public static final Assignment EMPTY = new Assignment( - (byte) 0, - Collections.emptyMap(), - VersionedMetadata.EMPTY - ); - - /** - * The error assigned to the member. - */ - private final byte error; + public static final Assignment EMPTY = new Assignment(Collections.emptyMap()); /** * The partitions assigned to the member. */ private final Map> partitions; - /** - * The metadata assigned to the member. - */ - private final VersionedMetadata metadata; - public Assignment( Map> partitions ) { - this( - (byte) 0, - partitions, - VersionedMetadata.EMPTY - ); - } - - public Assignment( - byte error, - Map> partitions, - VersionedMetadata metadata - ) { - this.error = error; this.partitions = Collections.unmodifiableMap(Objects.requireNonNull(partitions)); - this.metadata = Objects.requireNonNull(metadata); - } - - /** - * @return The error. - */ - public byte error() { - return error; } /** @@ -86,40 +50,22 @@ public Map> partitions() { return partitions; } - /** - * @return The metadata. - */ - public VersionedMetadata metadata() { - return metadata; - } - @Override public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - Assignment that = (Assignment) o; - - if (error != that.error) return false; - if (!partitions.equals(that.partitions)) return false; - return metadata.equals(that.metadata); + return partitions.equals(that.partitions); } @Override public int hashCode() { - int result = error; - result = 31 * result + partitions.hashCode(); - result = 31 * result + metadata.hashCode(); - return result; + return partitions.hashCode(); } @Override public String toString() { - return "Assignment(" + - "error=" + error + - ", partitions=" + partitions + - ", metadata=" + metadata + - ')'; + return "Assignment(partitions=" + partitions + ')'; } /** @@ -132,13 +78,9 @@ public static Assignment fromRecord( ConsumerGroupTargetAssignmentMemberValue record ) { return new Assignment( - record.error(), record.topicPartitions().stream().collect(Collectors.toMap( ConsumerGroupTargetAssignmentMemberValue.TopicPartition::topicId, - topicPartitions -> new HashSet<>(topicPartitions.partitions()))), - new VersionedMetadata( - record.metadataVersion(), - ByteBuffer.wrap(record.metadataBytes())) + topicPartitions -> new HashSet<>(topicPartitions.partitions()))) ); } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ClientAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ClientAssignor.java deleted file mode 100644 index fed73cd58d..0000000000 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ClientAssignor.java +++ /dev/null @@ -1,164 +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.coordinator.group.consumer; - -import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; - -import java.nio.ByteBuffer; -import java.util.Objects; - -/** - * An immutable representation of a client side assignor within a consumer group member. - */ -public class ClientAssignor { - /** - * The name of the assignor. - */ - private final String name; - - /** - * The reason reported by the assignor. - */ - private final byte reason; - - /** - * The minimum metadata version supported by the assignor. - */ - private final short minimumVersion; - - /** - * The maximum metadata version supported by the assignor. - */ - private final short maximumVersion; - - /** - * The versioned metadata. - */ - private final VersionedMetadata metadata; - - public ClientAssignor( - String name, - byte reason, - short minimumVersion, - short maximumVersion, - VersionedMetadata metadata - ) { - this.name = Objects.requireNonNull(name); - if (name.isEmpty()) { - throw new IllegalArgumentException("Assignor name cannot be empty."); - } - this.reason = reason; - this.minimumVersion = minimumVersion; - if (minimumVersion < -1) { - // -1 is supported as part of the upgrade from the old protocol to the new protocol. It - // basically means that the assignor supports metadata from the old client assignor. - throw new IllegalArgumentException("Assignor minimum version must be greater than -1."); - } - this.maximumVersion = maximumVersion; - if (maximumVersion < 0) { - throw new IllegalArgumentException("Assignor maximum version must be greater than or equals to 0."); - } else if (maximumVersion < minimumVersion) { - throw new IllegalArgumentException("Assignor maximum version must be greater than or equals to " - + "the minimum version."); - } - this.metadata = Objects.requireNonNull(metadata); - } - - /** - * @return The client side assignor name. - */ - public String name() { - return this.name; - } - - /** - * @return The current reason reported by the assignor. - */ - public byte reason() { - return this.reason; - } - - /** - * @return The minimum version supported by the assignor. - */ - public short minimumVersion() { - return this.minimumVersion; - } - - /** - * @return The maximum version supported by the assignor. - */ - public short maximumVersion() { - return this.maximumVersion; - } - - /** - * @return The versioned metadata. - */ - public VersionedMetadata metadata() { - return metadata; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - ClientAssignor that = (ClientAssignor) o; - - if (reason != that.reason) return false; - if (minimumVersion != that.minimumVersion) return false; - if (maximumVersion != that.maximumVersion) return false; - if (!name.equals(that.name)) return false; - return metadata.equals(that.metadata); - } - - @Override - public int hashCode() { - int result = name.hashCode(); - result = 31 * result + (int) reason; - result = 31 * result + (int) minimumVersion; - result = 31 * result + (int) maximumVersion; - result = 31 * result + metadata.hashCode(); - return result; - } - - @Override - public String toString() { - return "ClientAssignor(name=" + name + - ", reason=" + reason + - ", minimumVersion=" + minimumVersion + - ", maximumVersion=" + maximumVersion + - ", metadata=" + metadata + - ')'; - } - - public static ClientAssignor fromRecord( - ConsumerGroupMemberMetadataValue.Assignor record - ) { - return new ClientAssignor( - record.name(), - record.reason(), - record.minimumVersion(), - record.maximumVersion(), - new VersionedMetadata( - record.version(), - ByteBuffer.wrap(record.metadata()) - ) - ); - } -} diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java index d4077ef99d..68851b5176 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java @@ -779,7 +779,7 @@ private void maybeUpdateGroupState() { newState = ASSIGNING; } else { for (ConsumerGroupMember member : members.values()) { - if (member.targetMemberEpoch() != targetAssignmentEpoch.get() || member.state() != ConsumerGroupMember.MemberState.STABLE) { + if (!member.isReconciledTo(targetAssignmentEpoch.get())) { newState = RECONCILING; break; } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java index 5159f1f27c..3bf87caef9 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java @@ -41,6 +41,7 @@ * by records stored in the __consumer_offsets topic. */ public class ConsumerGroupMember { + /** * A builder that facilitates the creation of a new member or the update of * an existing one. @@ -52,7 +53,7 @@ public static class Builder { private final String memberId; private int memberEpoch = 0; private int previousMemberEpoch = -1; - private int targetMemberEpoch = 0; + private MemberState state = MemberState.STABLE; private String instanceId = null; private String rackId = null; private int rebalanceTimeoutMs = -1; @@ -61,10 +62,8 @@ public static class Builder { private List subscribedTopicNames = Collections.emptyList(); private String subscribedTopicRegex = ""; private String serverAssignorName = null; - private List clientAssignors = Collections.emptyList(); private Map> assignedPartitions = Collections.emptyMap(); private Map> partitionsPendingRevocation = Collections.emptyMap(); - private Map> partitionsPendingAssignment = Collections.emptyMap(); public Builder(String memberId) { this.memberId = Objects.requireNonNull(memberId); @@ -76,7 +75,6 @@ public Builder(ConsumerGroupMember member) { this.memberId = member.memberId; this.memberEpoch = member.memberEpoch; this.previousMemberEpoch = member.previousMemberEpoch; - this.targetMemberEpoch = member.targetMemberEpoch; this.instanceId = member.instanceId; this.rackId = member.rackId; this.rebalanceTimeoutMs = member.rebalanceTimeoutMs; @@ -85,24 +83,25 @@ public Builder(ConsumerGroupMember member) { this.subscribedTopicNames = member.subscribedTopicNames; this.subscribedTopicRegex = member.subscribedTopicRegex; this.serverAssignorName = member.serverAssignorName; - this.clientAssignors = member.clientAssignors; + this.state = member.state; this.assignedPartitions = member.assignedPartitions; this.partitionsPendingRevocation = member.partitionsPendingRevocation; - this.partitionsPendingAssignment = member.partitionsPendingAssignment; } - public Builder setMemberEpoch(int memberEpoch) { + public Builder updateMemberEpoch(int memberEpoch) { + int currentMemberEpoch = this.memberEpoch; this.memberEpoch = memberEpoch; + this.previousMemberEpoch = currentMemberEpoch; return this; } - public Builder setPreviousMemberEpoch(int previousMemberEpoch) { - this.previousMemberEpoch = previousMemberEpoch; + public Builder setMemberEpoch(int memberEpoch) { + this.memberEpoch = memberEpoch; return this; } - public Builder setTargetMemberEpoch(int targetMemberEpoch) { - this.targetMemberEpoch = targetMemberEpoch; + public Builder setPreviousMemberEpoch(int previousMemberEpoch) { + this.previousMemberEpoch = previousMemberEpoch; return this; } @@ -178,13 +177,8 @@ public Builder maybeUpdateServerAssignorName(Optional serverAssignorName return this; } - public Builder setClientAssignors(List clientAssignors) { - this.clientAssignors = clientAssignors; - return this; - } - - public Builder maybeUpdateClientAssignors(Optional> clientAssignors) { - this.clientAssignors = clientAssignors.orElse(this.clientAssignors); + public Builder setState(MemberState state) { + this.state = state; return this; } @@ -198,11 +192,6 @@ public Builder setPartitionsPendingRevocation(Map> partitions return this; } - public Builder setPartitionsPendingAssignment(Map> partitionsPendingAssignment) { - this.partitionsPendingAssignment = partitionsPendingAssignment; - return this; - } - public Builder updateWith(ConsumerGroupMemberMetadataValue record) { setInstanceId(record.instanceId()); setRackId(record.rackId()); @@ -212,19 +201,15 @@ public Builder updateWith(ConsumerGroupMemberMetadataValue record) { setSubscribedTopicRegex(record.subscribedTopicRegex()); setRebalanceTimeoutMs(record.rebalanceTimeoutMs()); setServerAssignorName(record.serverAssignor()); - setClientAssignors(record.assignors().stream() - .map(ClientAssignor::fromRecord) - .collect(Collectors.toList())); return this; } public Builder updateWith(ConsumerGroupCurrentMemberAssignmentValue record) { setMemberEpoch(record.memberEpoch()); setPreviousMemberEpoch(record.previousMemberEpoch()); - setTargetMemberEpoch(record.targetMemberEpoch()); + setState(MemberState.fromValue(record.state())); setAssignedPartitions(assignmentFromTopicPartitions(record.assignedPartitions())); setPartitionsPendingRevocation(assignmentFromTopicPartitions(record.partitionsPendingRevocation())); - setPartitionsPendingAssignment(assignmentFromTopicPartitions(record.partitionsPendingAssignment())); return this; } @@ -237,20 +222,10 @@ private Map> assignmentFromTopicPartitions( } public ConsumerGroupMember build() { - MemberState state; - if (!partitionsPendingRevocation.isEmpty()) { - state = MemberState.REVOKING; - } else if (!partitionsPendingAssignment.isEmpty()) { - state = MemberState.ASSIGNING; - } else { - state = MemberState.STABLE; - } - return new ConsumerGroupMember( memberId, memberEpoch, previousMemberEpoch, - targetMemberEpoch, instanceId, rackId, rebalanceTimeoutMs, @@ -259,36 +234,13 @@ public ConsumerGroupMember build() { subscribedTopicNames, subscribedTopicRegex, serverAssignorName, - clientAssignors, state, assignedPartitions, - partitionsPendingRevocation, - partitionsPendingAssignment + partitionsPendingRevocation ); } } - /** - * The various states that a member can be in. For their definition, - * refer to the documentation of {{@link CurrentAssignmentBuilder}}. - */ - public enum MemberState { - REVOKING("revoking"), - ASSIGNING("assigning"), - STABLE("stable"); - - private final String name; - - MemberState(String name) { - this.name = name; - } - - @Override - public String toString() { - return name; - } - } - /** * The member id. */ @@ -305,11 +257,9 @@ public String toString() { private final int previousMemberEpoch; /** - * The next member epoch. This corresponds to the target - * assignment epoch used to compute the current assigned, - * revoking and assigning partitions. + * The member state. */ - private final int targetMemberEpoch; + private final MemberState state; /** * The instance id provided by the member. @@ -351,16 +301,6 @@ public String toString() { */ private final String serverAssignorName; - /** - * The states of the client side assignors of the member. - */ - private final List clientAssignors; - - /** - * The member state. - */ - private final MemberState state; - /** * The partitions assigned to this member. */ @@ -371,18 +311,10 @@ public String toString() { */ private final Map> partitionsPendingRevocation; - /** - * The partitions waiting to be assigned to this - * member. They will be assigned when they are - * released by their previous owners. - */ - private final Map> partitionsPendingAssignment; - private ConsumerGroupMember( String memberId, int memberEpoch, int previousMemberEpoch, - int targetMemberEpoch, String instanceId, String rackId, int rebalanceTimeoutMs, @@ -391,16 +323,14 @@ private ConsumerGroupMember( List subscribedTopicNames, String subscribedTopicRegex, String serverAssignorName, - List clientAssignors, MemberState state, Map> assignedPartitions, - Map> partitionsPendingRevocation, - Map> partitionsPendingAssignment + Map> partitionsPendingRevocation ) { this.memberId = memberId; this.memberEpoch = memberEpoch; this.previousMemberEpoch = previousMemberEpoch; - this.targetMemberEpoch = targetMemberEpoch; + this.state = state; this.instanceId = instanceId; this.rackId = rackId; this.rebalanceTimeoutMs = rebalanceTimeoutMs; @@ -409,11 +339,8 @@ private ConsumerGroupMember( this.subscribedTopicNames = subscribedTopicNames; this.subscribedTopicRegex = subscribedTopicRegex; this.serverAssignorName = serverAssignorName; - this.clientAssignors = clientAssignors; - this.state = state; this.assignedPartitions = assignedPartitions; this.partitionsPendingRevocation = partitionsPendingRevocation; - this.partitionsPendingAssignment = partitionsPendingAssignment; } /** @@ -437,13 +364,6 @@ public int previousMemberEpoch() { return previousMemberEpoch; } - /** - * @return The target member epoch. - */ - public int targetMemberEpoch() { - return targetMemberEpoch; - } - /** * @return The instance id. */ @@ -501,17 +421,17 @@ public Optional serverAssignorName() { } /** - * @return The list of client side assignors. + * @return The current state. */ - public List clientAssignors() { - return clientAssignors; + public MemberState state() { + return state; } /** - * @return The current state. + * @return True if the member is in the Stable state and at the desired epoch. */ - public MemberState state() { - return state; + public boolean isReconciledTo(int targetAssignmentEpoch) { + return state == MemberState.STABLE && memberEpoch == targetAssignmentEpoch; } /** @@ -528,27 +448,6 @@ public Map> partitionsPendingRevocation() { return partitionsPendingRevocation; } - /** - * @return The set of partitions awaiting assignment to the member. - */ - public Map> partitionsPendingAssignment() { - return partitionsPendingAssignment; - } - - /** - * @return A string representation of the current assignment state. - */ - public String currentAssignmentSummary() { - return "CurrentAssignment(memberEpoch=" + memberEpoch + - ", previousMemberEpoch=" + previousMemberEpoch + - ", targetMemberEpoch=" + targetMemberEpoch + - ", state=" + state + - ", assignedPartitions=" + assignedPartitions + - ", partitionsPendingRevocation=" + partitionsPendingRevocation + - ", partitionsPendingAssignment=" + partitionsPendingAssignment + - ')'; - } - /** * @param targetAssignment The target assignment of this member in the corresponding group. * @@ -612,7 +511,7 @@ public boolean equals(Object o) { ConsumerGroupMember that = (ConsumerGroupMember) o; return memberEpoch == that.memberEpoch && previousMemberEpoch == that.previousMemberEpoch - && targetMemberEpoch == that.targetMemberEpoch + && state == that.state && rebalanceTimeoutMs == that.rebalanceTimeoutMs && Objects.equals(memberId, that.memberId) && Objects.equals(instanceId, that.instanceId) @@ -622,10 +521,8 @@ public boolean equals(Object o) { && Objects.equals(subscribedTopicNames, that.subscribedTopicNames) && Objects.equals(subscribedTopicRegex, that.subscribedTopicRegex) && Objects.equals(serverAssignorName, that.serverAssignorName) - && Objects.equals(clientAssignors, that.clientAssignors) && Objects.equals(assignedPartitions, that.assignedPartitions) - && Objects.equals(partitionsPendingRevocation, that.partitionsPendingRevocation) - && Objects.equals(partitionsPendingAssignment, that.partitionsPendingAssignment); + && Objects.equals(partitionsPendingRevocation, that.partitionsPendingRevocation); } @Override @@ -633,7 +530,7 @@ public int hashCode() { int result = memberId != null ? memberId.hashCode() : 0; result = 31 * result + memberEpoch; result = 31 * result + previousMemberEpoch; - result = 31 * result + targetMemberEpoch; + result = 31 * result + Objects.hashCode(state); result = 31 * result + Objects.hashCode(instanceId); result = 31 * result + Objects.hashCode(rackId); result = 31 * result + rebalanceTimeoutMs; @@ -642,10 +539,8 @@ public int hashCode() { result = 31 * result + Objects.hashCode(subscribedTopicNames); result = 31 * result + Objects.hashCode(subscribedTopicRegex); result = 31 * result + Objects.hashCode(serverAssignorName); - result = 31 * result + Objects.hashCode(clientAssignors); result = 31 * result + Objects.hashCode(assignedPartitions); result = 31 * result + Objects.hashCode(partitionsPendingRevocation); - result = 31 * result + Objects.hashCode(partitionsPendingAssignment); return result; } @@ -655,7 +550,7 @@ public String toString() { "memberId='" + memberId + '\'' + ", memberEpoch=" + memberEpoch + ", previousMemberEpoch=" + previousMemberEpoch + - ", targetMemberEpoch=" + targetMemberEpoch + + ", state='" + state + '\'' + ", instanceId='" + instanceId + '\'' + ", rackId='" + rackId + '\'' + ", rebalanceTimeoutMs=" + rebalanceTimeoutMs + @@ -664,11 +559,18 @@ public String toString() { ", subscribedTopicNames=" + subscribedTopicNames + ", subscribedTopicRegex='" + subscribedTopicRegex + '\'' + ", serverAssignorName='" + serverAssignorName + '\'' + - ", clientAssignors=" + clientAssignors + - ", state=" + state + ", assignedPartitions=" + assignedPartitions + ", partitionsPendingRevocation=" + partitionsPendingRevocation + - ", partitionsPendingAssignment=" + partitionsPendingAssignment + ')'; } + + /** + * @return True of the two provided members have different assigned partitions. + */ + public static boolean hasAssignedPartitionsChanged( + ConsumerGroupMember member1, + ConsumerGroupMember member2 + ) { + return !member1.assignedPartitions().equals(member2.assignedPartitions()); + } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java index fce5b8a85b..f4306d95fc 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java @@ -17,12 +17,12 @@ package org.apache.kafka.coordinator.group.consumer; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.FencedMemberEpochException; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -33,49 +33,6 @@ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the * consumer group protocol. Given the current state of a member and a desired or target * assignment state, the state machine takes the necessary steps to converge them. - * - * The member state has the following properties: - * - Current Epoch: - * The current epoch of the member. - * - * - Next Epoch: - * The desired epoch of the member. It corresponds to the epoch of the target/desired assignment. - * The member transitions to this epoch when it has revoked the partitions that it does not own - * or if it does not have to revoke any. - * - * - Previous Epoch: - * The epoch of the member when the state was last updated. - * - * - Assigned Partitions: - * The set of partitions currently assigned to the member. This represents what the member should have. - * - * - Partitions Pending Revocation: - * The set of partitions that the member should revoke before it can transition to the next state. - * - * - Partitions Pending Assignment: - * The set of partitions that the member will eventually receive. The partitions in this set are - * still owned by other members in the group. - * - * The state machine has three states: - * - REVOKING: - * This state means that the member must revoke partitions before it can transition to the next epoch - * and thus start receiving new partitions. This is to guarantee that offsets of revoked partitions - * are committed with the current epoch. The member transitions to the next state only when it has - * acknowledged the revocation. - * - * - ASSIGNING: - * This state means that the member waits on partitions which are still owned by other members in the - * group. It remains in this state until they are all freed up. - * - * - STABLE: - * This state means that the member has received all its assigned partitions. - * - * The reconciliation process is started or re-started whenever a new target assignment is installed; - * the epoch of the new target assignment is different from the next epoch of the member. In this transient - * state, the assigned partitions, the partitions pending revocation and the partitions pending assignment - * are updated. If the partitions pending revocation is not empty, the state machine transitions to - * REVOKING; if partitions pending assignment is not empty, it transitions to ASSIGNING; otherwise it - * transitions to STABLE. */ public class CurrentAssignmentBuilder { /** @@ -170,72 +127,122 @@ public CurrentAssignmentBuilder withOwnedTopicPartitions( * @return A new ConsumerGroupMember or the current one. */ public ConsumerGroupMember build() { - // A new target assignment has been installed, we need to restart - // the reconciliation loop from the beginning. - if (targetAssignmentEpoch != member.targetMemberEpoch()) { - return transitionToNewTargetAssignmentState(); - } - switch (member.state()) { - // Check if the partitions have been revoked by the member. - case REVOKING: - return maybeTransitionFromRevokingToAssigningOrStable(); + case STABLE: + // When the member is in the STABLE state, we verify if a newer + // epoch (or target assignment) is available. If it is, we can + // reconcile the member towards it. Otherwise, we return. + if (member.memberEpoch() != targetAssignmentEpoch) { + return computeNextAssignment( + member.memberEpoch(), + member.assignedPartitions() + ); + } else { + return member; + } - // Check if pending partitions have been freed up. - case ASSIGNING: - return maybeTransitionFromAssigningToAssigningOrStable(); + case UNREVOKED_PARTITIONS: + // When the member is in the UNREVOKED_PARTITIONS state, we wait + // until the member has revoked the necessary partitions. They are + // considered revoked when they are not anymore reported in the + // owned partitions set in the ConsumerGroupHeartbeat API. - // Nothing to do. - case STABLE: - return member; + // If the member does not provide its owned partitions. We cannot + // progress. + if (ownedTopicPartitions == null) { + return member; + } + + // If the member provides its owned partitions. We verify if it still + // owns any of the revoked partitions. If it does, we cannot progress. + for (ConsumerGroupHeartbeatRequestData.TopicPartitions topicPartitions : ownedTopicPartitions) { + for (Integer partitionId : topicPartitions.partitions()) { + boolean stillHasRevokedPartition = member + .partitionsPendingRevocation() + .getOrDefault(topicPartitions.topicId(), Collections.emptySet()) + .contains(partitionId); + if (stillHasRevokedPartition) { + return member; + } + } + } + + // When the member has revoked all the pending partitions, it can + // transition to the next epoch (current + 1) and we can reconcile + // its state towards the latest target assignment. + return computeNextAssignment( + member.memberEpoch() + 1, + member.assignedPartitions() + ); + + case UNRELEASED_PARTITIONS: + // When the member is in the UNRELEASED_PARTITIONS, we reconcile the + // member towards the latest target assignment. This will assign any + // of the unreleased partitions when they become available. + return computeNextAssignment( + member.memberEpoch(), + member.assignedPartitions() + ); + + case UNKNOWN: + // We could only end up in this state if a new state is added in the + // future and the group coordinator is downgraded. In this case, the + // best option is to fence the member to force it to rejoin the group + // without any partitions and to reconcile it again from scratch. + if (ownedTopicPartitions == null || !ownedTopicPartitions.isEmpty()) { + throw new FencedMemberEpochException("The consumer group member is in a unknown state. " + + "The member must abandon all its partitions and rejoin."); + } + + return computeNextAssignment( + targetAssignmentEpoch, + member.assignedPartitions() + ); } return member; } /** - * Transitions to NewTargetAssignment state. This is a transient state where - * we compute the assigned partitions, the partitions pending revocation, - * the partitions pending assignment, and transition to the next state. + * Computes the next assignment. * + * @param memberEpoch The epoch of the member to use. This may be different + * from the epoch in {@link CurrentAssignmentBuilder#member}. + * @param memberAssignedPartitions The assigned partitions of the member to use. * @return A new ConsumerGroupMember. */ - private ConsumerGroupMember transitionToNewTargetAssignmentState() { + private ConsumerGroupMember computeNextAssignment( + int memberEpoch, + Map> memberAssignedPartitions + ) { + boolean hasUnreleasedPartitions = false; Map> newAssignedPartitions = new HashMap<>(); Map> newPartitionsPendingRevocation = new HashMap<>(); Map> newPartitionsPendingAssignment = new HashMap<>(); - // Compute the combined set of topics. Set allTopicIds = new HashSet<>(targetAssignment.partitions().keySet()); - allTopicIds.addAll(member.assignedPartitions().keySet()); - allTopicIds.addAll(member.partitionsPendingRevocation().keySet()); - allTopicIds.addAll(member.partitionsPendingAssignment().keySet()); + allTopicIds.addAll(memberAssignedPartitions.keySet()); for (Uuid topicId : allTopicIds) { Set target = targetAssignment.partitions() .getOrDefault(topicId, Collections.emptySet()); - Set currentAssignedPartitions = member.assignedPartitions() - .getOrDefault(topicId, Collections.emptySet()); - Set currentRevokingPartitions = member.partitionsPendingRevocation() + Set currentAssignedPartitions = memberAssignedPartitions .getOrDefault(topicId, Collections.emptySet()); - // Assigned_1 = (Assigned_0 + Pending_Revocation_0) ∩ Target - // Assigned_0 + Pending_Revocation_0 is used here because the partitions - // being revoked are still owned until the revocation is acknowledged. + // New Assigned Partitions = Previous Assigned Partitions ∩ Target Set assignedPartitions = new HashSet<>(currentAssignedPartitions); - assignedPartitions.addAll(currentRevokingPartitions); assignedPartitions.retainAll(target); - // Pending_Revocation_1 = (Assigned_0 + Pending_Revocation_0) - Assigned_1 - // Assigned_0 + Pending_Revocation_0 is used here because the partitions - // being revoked are still owned until the revocation is acknowledged. + // Partitions Pending Revocation = Previous Assigned Partitions - New Assigned Partitions Set partitionsPendingRevocation = new HashSet<>(currentAssignedPartitions); - partitionsPendingRevocation.addAll(currentRevokingPartitions); partitionsPendingRevocation.removeAll(assignedPartitions); - // Pending_Assignment_1 = Target - Assigned_1 + // Partitions Pending Assignment = Target - New Assigned Partitions - Unreleased Partitions Set partitionsPendingAssignment = new HashSet<>(target); partitionsPendingAssignment.removeAll(assignedPartitions); + hasUnreleasedPartitions = partitionsPendingAssignment.removeIf(partitionId -> + currentPartitionEpoch.apply(topicId, partitionId) != -1 + ) || hasUnreleasedPartitions; if (!assignedPartitions.isEmpty()) { newAssignedPartitions.put(topicId, assignedPartitions); @@ -251,195 +258,51 @@ private ConsumerGroupMember transitionToNewTargetAssignmentState() { } if (!newPartitionsPendingRevocation.isEmpty()) { - // If the partition pending revocation set is not empty, we transition the - // member to revoking and keep the current epoch. The transition to the new - // state is done when the member is updated. + // If there are partitions to be revoked, the member remains in its current + // epoch and requests the revocation of those partitions. It transitions to + // the UNREVOKED_PARTITIONS state to wait until the client acknowledges the + // revocation of the partitions. return new ConsumerGroupMember.Builder(member) + .setState(MemberState.UNREVOKED_PARTITIONS) + .updateMemberEpoch(memberEpoch) .setAssignedPartitions(newAssignedPartitions) .setPartitionsPendingRevocation(newPartitionsPendingRevocation) - .setPartitionsPendingAssignment(newPartitionsPendingAssignment) - .setTargetMemberEpoch(targetAssignmentEpoch) .build(); - } else { - if (!newPartitionsPendingAssignment.isEmpty()) { - // If the partitions pending assignment set is not empty, we check - // if some or all partitions are free to use. If they are, we move - // them to the partitions assigned set. - maybeAssignPendingPartitions(newAssignedPartitions, newPartitionsPendingAssignment); - } - - // We transition to the target epoch. If the partitions pending assignment - // set is empty, the member transition to stable, otherwise to assigning. - // The transition to the new state is done when the member is updated. + } else if (!newPartitionsPendingAssignment.isEmpty()) { + // If there are partitions to be assigned, the member transitions to the + // target epoch and requests the assignment of those partitions. Note that + // the partitions are directly added to the assigned partitions set. The + // member transitions to the STABLE state or to the UNRELEASED_PARTITIONS + // state depending on whether there are unreleased partitions or not. + newPartitionsPendingAssignment.forEach((topicId, partitions) -> newAssignedPartitions + .computeIfAbsent(topicId, __ -> new HashSet<>()) + .addAll(partitions)); + MemberState newState = hasUnreleasedPartitions ? MemberState.UNRELEASED_PARTITIONS : MemberState.STABLE; return new ConsumerGroupMember.Builder(member) + .setState(newState) + .updateMemberEpoch(targetAssignmentEpoch) .setAssignedPartitions(newAssignedPartitions) .setPartitionsPendingRevocation(Collections.emptyMap()) - .setPartitionsPendingAssignment(newPartitionsPendingAssignment) - .setPreviousMemberEpoch(member.memberEpoch()) - .setMemberEpoch(targetAssignmentEpoch) - .setTargetMemberEpoch(targetAssignmentEpoch) .build(); - } - } - - /** - * Tries to transition from Revoke to Assigning or Stable. This is only - * possible when the member acknowledges that it only owns the partition - * in the assigned partitions. - * - * @return A new ConsumerGroupMember with the new state or the current one - * if the member stays in the current state. - */ - private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() { - if (member.partitionsPendingRevocation().isEmpty() || matchesAssignedPartitions(ownedTopicPartitions)) { - Map> newAssignedPartitions = deepCopy(member.assignedPartitions()); - Map> newPartitionsPendingAssignment = deepCopy(member.partitionsPendingAssignment()); - - if (!newPartitionsPendingAssignment.isEmpty()) { - // If the partitions pending assignment set is not empty, we check - // if some or all partitions are free to use. If they are, we move - // them to the assigned set. - maybeAssignPendingPartitions(newAssignedPartitions, newPartitionsPendingAssignment); - } - - // We transition to the target epoch. If the partitions pending assignment - // set is empty, the member transition to stable, otherwise to assigning. - // The transition to the new state is done when the member is updated. + } else if (hasUnreleasedPartitions) { + // If there are no partitions to be revoked nor to be assigned but some + // partitions are not available yet, the member transitions to the target + // epoch, to the UNRELEASED_PARTITIONS state and waits. return new ConsumerGroupMember.Builder(member) + .setState(MemberState.UNRELEASED_PARTITIONS) + .updateMemberEpoch(targetAssignmentEpoch) .setAssignedPartitions(newAssignedPartitions) .setPartitionsPendingRevocation(Collections.emptyMap()) - .setPartitionsPendingAssignment(newPartitionsPendingAssignment) - .setPreviousMemberEpoch(member.memberEpoch()) - .setMemberEpoch(targetAssignmentEpoch) - .setTargetMemberEpoch(targetAssignmentEpoch) .build(); } else { - return member; - } - } - - /** - * Tries to transition from Assigning to Assigning or Stable. This is only - * possible when one or more partitions in the partitions pending assignment - * set have been freed up by other members in the group. - * - * @return A new ConsumerGroupMember with the new state or the current one - * if the member stays in the current state. - */ - private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() { - Map> newAssignedPartitions = deepCopy(member.assignedPartitions()); - Map> newPartitionsPendingAssignment = deepCopy(member.partitionsPendingAssignment()); - - // If any partition can transition from assigning to assigned, we update - // the member. Otherwise, we return the current one. The transition to the - // new state is done when the member is updated. - if (maybeAssignPendingPartitions(newAssignedPartitions, newPartitionsPendingAssignment)) { + // Otherwise, the member transitions to the target epoch and to the + // STABLE state. return new ConsumerGroupMember.Builder(member) + .setState(MemberState.STABLE) + .updateMemberEpoch(targetAssignmentEpoch) .setAssignedPartitions(newAssignedPartitions) .setPartitionsPendingRevocation(Collections.emptyMap()) - .setPartitionsPendingAssignment(newPartitionsPendingAssignment) - .setPreviousMemberEpoch(member.memberEpoch()) - .setMemberEpoch(targetAssignmentEpoch) - .setTargetMemberEpoch(targetAssignmentEpoch) .build(); - } else { - return member; - } - } - - /** - * Tries to move partitions from the partitions pending assignment set to - * the partitions assigned set if they are no longer owned. - * - * @param newAssignedPartitions The assigned partitions. - * @param newPartitionsPendingAssignment The partitions pending assignment. - * @return A boolean indicating if any partitions were moved. - */ - private boolean maybeAssignPendingPartitions( - Map> newAssignedPartitions, - Map> newPartitionsPendingAssignment - ) { - boolean changed = false; - - Iterator>> assigningSetIterator = - newPartitionsPendingAssignment.entrySet().iterator(); - - while (assigningSetIterator.hasNext()) { - Map.Entry> pair = assigningSetIterator.next(); - Uuid topicId = pair.getKey(); - Set assigning = pair.getValue(); - - Iterator assigningIterator = assigning.iterator(); - while (assigningIterator.hasNext()) { - Integer partitionId = assigningIterator.next(); - - // A partition can be assigned to this member iff it has been - // released by its previous owner. This is signaled by -1. - Integer partitionEpoch = currentPartitionEpoch.apply(topicId, partitionId); - if (partitionEpoch == -1) { - assigningIterator.remove(); - put(newAssignedPartitions, topicId, partitionId); - changed = true; - } - } - - if (assigning.isEmpty()) { - assigningSetIterator.remove(); - } - } - - return changed; - } - - /** - * Checks whether the owned topic partitions passed by the member to the state - * machine via the ConsumerGroupHeartbeat request corresponds to the assigned - * partitions. - * - * @param ownedTopicPartitions The topic partitions owned by the remove client. - * @return A boolean indicating if the owned partitions matches the Assigned set. - */ - private boolean matchesAssignedPartitions( - List ownedTopicPartitions - ) { - if (ownedTopicPartitions == null) return false; - if (ownedTopicPartitions.size() != member.assignedPartitions().size()) return false; - - for (ConsumerGroupHeartbeatRequestData.TopicPartitions topicPartitions : ownedTopicPartitions) { - Set partitions = member.assignedPartitions().get(topicPartitions.topicId()); - if (partitions == null) return false; - for (Integer partitionId : topicPartitions.partitions()) { - if (!partitions.contains(partitionId)) return false; - } } - - return true; - } - - /** - * Makes a deep copy of an assignment map. - * - * @param map The Map to copy. - * @return The copy. - */ - private Map> deepCopy(Map> map) { - Map> copy = new HashMap<>(); - map.forEach((topicId, partitions) -> copy.put(topicId, new HashSet<>(partitions))); - return copy; - } - - /** - * Puts the given TopicId and Partitions to the given map. - */ - private void put( - Map> map, - Uuid topicId, - Integer partitionId - ) { - map.compute(topicId, (__, partitionsOrNull) -> { - if (partitionsOrNull == null) partitionsOrNull = new HashSet<>(); - partitionsOrNull.add(partitionId); - return partitionsOrNull; - }); } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/MemberState.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/MemberState.java new file mode 100644 index 0000000000..3f3237bfe0 --- /dev/null +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/MemberState.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.group.consumer; + +import java.util.HashMap; +import java.util.Map; + +/** + * The various states that a member can be in. For their definition, + * refer to the documentation of {{@link CurrentAssignmentBuilder}}. + */ +public enum MemberState { + /** + * The member is fully reconciled with the desired target assignment. + */ + STABLE((byte) 0), + + /** + * The member must revoke some partitions in order to be able to + * transition to the next epoch. + */ + UNREVOKED_PARTITIONS((byte) 1), + + /** + * The member transitioned to the last epoch but waits on some + * partitions which have not been revoked by their previous + * owners yet. + */ + UNRELEASED_PARTITIONS((byte) 2), + + /** + * The member is in an unknown state. This can only happen if a future + * version of the software introduces a new state unknown by this version. + */ + UNKNOWN((byte) 127); + + private final static Map VALUES_TO_ENUMS = new HashMap<>(); + + static { + for (MemberState state: MemberState.values()) { + VALUES_TO_ENUMS.put(state.value(), state); + } + } + + private final byte value; + + MemberState(byte value) { + this.value = value; + } + + public byte value() { + return value; + } + + public static MemberState fromValue(byte value) { + MemberState state = VALUES_TO_ENUMS.get(value); + if (state == null) { + return UNKNOWN; + } + return state; + } +} diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/VersionedMetadata.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/VersionedMetadata.java deleted file mode 100644 index 89baeb82bf..0000000000 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/VersionedMetadata.java +++ /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 org.apache.kafka.coordinator.group.consumer; - -import java.nio.ByteBuffer; -import java.util.Objects; - -/** - * Immutable versioned metadata. It contains a bunch of bytes tagged with a version. The - * format of the bytes is unspecified. This is mainly used by client side assignors to - * exchange arbitrary metadata between the members and the assignor and vice versa. - */ -public class VersionedMetadata { - public static final VersionedMetadata EMPTY = new VersionedMetadata((short) 0, ByteBuffer.allocate(0)); - - /** - * The version of the metadata encoded in {{@link VersionedMetadata#metadata}}. - */ - private final short version; - - /** - * The metadata bytes. - */ - private final ByteBuffer metadata; - - public VersionedMetadata( - short version, - ByteBuffer metadata - ) { - this.version = version; - this.metadata = Objects.requireNonNull(metadata); - } - - /** - * @return The version of the metadata. - */ - public short version() { - return this.version; - } - - /** - * @return The ByteBuffer holding the metadata. - */ - public ByteBuffer metadata() { - return this.metadata; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - VersionedMetadata that = (VersionedMetadata) o; - - if (version != that.version) return false; - return metadata.equals(that.metadata); - } - - @Override - public int hashCode() { - int result = version; - result = 31 * result + metadata.hashCode(); - return result; - } - - @Override - public String toString() { - return "VersionedMetadata(" + - "version=" + version + - ", metadata=" + metadata + - ')'; - } -} diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorEventProcessor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorEventProcessor.java index 0195880c4a..f2463a7cd7 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorEventProcessor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorEventProcessor.java @@ -24,10 +24,18 @@ public interface CoordinatorEventProcessor extends AutoCloseable { /** - * Enqueues a new {{@link CoordinatorEvent}}. + * Enqueues a new {{@link CoordinatorEvent}} at the end of the processor. * * @param event The event. * @throws RejectedExecutionException If the event processor is closed. */ - void enqueue(CoordinatorEvent event) throws RejectedExecutionException; + void enqueueLast(CoordinatorEvent event) throws RejectedExecutionException; + + /** + * Enqueues a new {{@link CoordinatorEvent}} at the front of the processor. + * + * @param event The event. + * @throws RejectedExecutionException If the event processor is closed. + */ + void enqueueFirst(CoordinatorEvent event) throws RejectedExecutionException; } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java index a2613ba837..2f52f1d11f 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java @@ -41,19 +41,20 @@ import java.time.Duration; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.OptionalInt; import java.util.OptionalLong; -import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; +import java.util.stream.Collectors; /** * The CoordinatorRuntime provides a framework to implement coordinators such as the group coordinator @@ -356,7 +357,7 @@ public void run() { log.debug("Scheduling write event {} for timer {}.", event.name, key); try { - enqueue(event); + enqueueLast(event); } catch (NotCoordinatorException ex) { log.info("Failed to enqueue write event {} for timer {} because the runtime is closed. Ignoring it.", event.name, key); @@ -438,6 +439,12 @@ class CoordinatorContext { */ SnapshottableCoordinator coordinator; + /** + * The high watermark listener registered to all the partitions + * backing the coordinators. + */ + HighWatermarkListener highWatermarklistener; + /** * Constructor. * @@ -495,6 +502,7 @@ private void transitionTo( case ACTIVE: state = CoordinatorState.ACTIVE; + highWatermarklistener = new HighWatermarkListener(); partitionWriter.registerListener(tp, highWatermarklistener); coordinator.onLoaded(metadataImage); break; @@ -520,7 +528,10 @@ private void transitionTo( * Unloads the coordinator. */ private void unload() { - partitionWriter.deregisterListener(tp, highWatermarklistener); + if (highWatermarklistener != null) { + partitionWriter.deregisterListener(tp, highWatermarklistener); + highWatermarklistener = null; + } timer.cancelAll(); deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception()); if (coordinator != null) { @@ -1179,6 +1190,23 @@ public String toString() { * backing the coordinator are updated. */ class HighWatermarkListener implements PartitionWriter.Listener { + + private static final long NO_OFFSET = -1L; + + /** + * The atomic long is used to store the last and unprocessed high watermark + * received from the partition. The atomic value is replaced by -1L when + * the high watermark is taken to update the context. + */ + private final AtomicLong lastHighWatermark = new AtomicLong(NO_OFFSET); + + /** + * @return The last high watermark received or NO_OFFSET is none is pending. + */ + public long lastHighWatermark() { + return lastHighWatermark.get(); + } + /** * Updates the high watermark of the corresponding coordinator. * @@ -1191,13 +1219,37 @@ public void onHighWatermarkUpdated( long offset ) { log.debug("High watermark of {} incremented to {}.", tp, offset); - scheduleInternalOperation("HighWatermarkUpdated(tp=" + tp + ", offset=" + offset + ")", tp, () -> { - withActiveContextOrThrow(tp, context -> { - context.coordinator.updateLastCommittedOffset(offset); - context.deferredEventQueue.completeUpTo(offset); - coordinatorMetrics.onUpdateLastCommittedOffset(tp, offset); - }); - }); + if (lastHighWatermark.getAndSet(offset) == NO_OFFSET) { + // An event to apply the new high watermark is pushed to the front of the + // queue only if the previous value was -1L. If it was not, it means that + // there is already an event waiting to process the last value. + enqueueFirst(new CoordinatorInternalEvent("HighWatermarkUpdate", tp, () -> { + long newHighWatermark = lastHighWatermark.getAndSet(NO_OFFSET); + + CoordinatorContext context = coordinators.get(tp); + if (context != null) { + context.lock.lock(); + try { + if (context.state == CoordinatorState.ACTIVE) { + // The updated high watermark can be applied to the coordinator only if the coordinator + // exists and is in the active state. + log.debug("Updating high watermark of {} to {}.", tp, newHighWatermark); + context.coordinator.updateLastCommittedOffset(newHighWatermark); + context.deferredEventQueue.completeUpTo(newHighWatermark); + coordinatorMetrics.onUpdateLastCommittedOffset(tp, newHighWatermark); + } else { + log.debug("Ignored high watermark updated for {} to {} because the coordinator is not active.", + tp, newHighWatermark); + } + } finally { + context.lock.unlock(); + } + } else { + log.debug("Ignored high watermark updated for {} to {} because the coordinator does not exist.", + tp, newHighWatermark); + } + })); + } } } @@ -1246,12 +1298,6 @@ public void onHighWatermarkUpdated( */ private final PartitionWriter partitionWriter; - /** - * The high watermark listener registered to all the partitions - * backing the coordinators. - */ - private final HighWatermarkListener highWatermarklistener; - /** * The coordinator loaded used by the runtime. */ @@ -1318,7 +1364,6 @@ private CoordinatorRuntime( this.coordinators = new ConcurrentHashMap<>(); this.processor = processor; this.partitionWriter = partitionWriter; - this.highWatermarklistener = new HighWatermarkListener(); this.loader = loader; this.coordinatorShardBuilderSupplier = coordinatorShardBuilderSupplier; this.runtimeMetrics = runtimeMetrics; @@ -1336,28 +1381,39 @@ private void throwIfNotRunning() { } /** - * Enqueues a new event. + * Enqueues a new event at the end of the processing queue. * * @param event The event. * @throws NotCoordinatorException If the event processor is closed. */ - private void enqueue(CoordinatorEvent event) { + private void enqueueLast(CoordinatorEvent event) { try { - processor.enqueue(event); + processor.enqueueLast(event); } catch (RejectedExecutionException ex) { throw new NotCoordinatorException("Can't accept an event because the processor is closed", ex); } } /** - * Creates the context if it does not exist. + * Enqueues a new event at the front of the processing queue. * - * @param tp The topic partition. - * - * Visible for testing. + * @param event The event. + * @throws NotCoordinatorException If the event processor is closed. */ - void maybeCreateContext(TopicPartition tp) { - coordinators.computeIfAbsent(tp, CoordinatorContext::new); + private void enqueueFirst(CoordinatorEvent event) { + try { + processor.enqueueFirst(event); + } catch (RejectedExecutionException ex) { + throw new NotCoordinatorException("Can't accept an event because the processor is closed", ex); + } + } + + /** + * @return The coordinator context or a new context if it does not exist. + * Package private for testing. + */ + CoordinatorContext maybeCreateContext(TopicPartition tp) { + return coordinators.computeIfAbsent(tp, CoordinatorContext::new); } /** @@ -1376,29 +1432,6 @@ CoordinatorContext contextOrThrow(TopicPartition tp) throws NotCoordinatorExcept } } - /** - * Calls the provided function with the context; throws an exception otherwise. - * This method ensures that the context lock is acquired before calling the - * function and releases afterwards. - * - * @param tp The topic partition. - * @param func The function that will receive the context. - * @throws NotCoordinatorException - */ - private void withContextOrThrow( - TopicPartition tp, - Consumer func - ) throws NotCoordinatorException { - CoordinatorContext context = contextOrThrow(tp); - - try { - context.lock.lock(); - func.accept(context); - } finally { - context.lock.unlock(); - } - } - /** * Calls the provided function with the context iff the context is active; throws * an exception otherwise. This method ensures that the context lock is acquired @@ -1451,10 +1484,36 @@ public CompletableFuture scheduleWriteOperation( throwIfNotRunning(); log.debug("Scheduled execution of write operation {}.", name); CoordinatorWriteEvent event = new CoordinatorWriteEvent<>(name, tp, timeout, op); - enqueue(event); + enqueueLast(event); return event.future; } + /** + * Schedule a write operation for each coordinator. + * + * @param name The name of the write operation. + * @param timeout The write operation timeout. + * @param op The write operation. + * + * @return A list of futures where each future will be completed with the result of the write operation + * when the operation is completed or an exception if the write operation failed. + * + * @param The type of the result. + */ + public List> scheduleWriteAllOperation( + String name, + Duration timeout, + CoordinatorWriteOperation op + ) { + throwIfNotRunning(); + log.debug("Scheduled execution of write all operation {}.", name); + return coordinators + .keySet() + .stream() + .map(tp -> scheduleWriteOperation(name, tp, timeout, op)) + .collect(Collectors.toList()); + } + /** * Schedules a transactional write operation. * @@ -1465,6 +1524,7 @@ public CompletableFuture scheduleWriteOperation( * @param producerEpoch The producer epoch. * @param timeout The write operation timeout. * @param op The write operation. + * @param apiVersion The Version of the Txn_Offset_Commit request * * @return A future that will be completed with the result of the write operation * when the operation is completed or an exception if the write operation failed. @@ -1478,7 +1538,8 @@ public CompletableFuture scheduleTransactionalWriteOperation( long producerId, short producerEpoch, Duration timeout, - CoordinatorWriteOperation op + CoordinatorWriteOperation op, + Short apiVersion ) { throwIfNotRunning(); log.debug("Scheduled execution of transactional write operation {}.", name); @@ -1486,7 +1547,8 @@ public CompletableFuture scheduleTransactionalWriteOperation( tp, transactionalId, producerId, - producerEpoch + producerEpoch, + apiVersion ).thenCompose(verificationGuard -> { CoordinatorWriteEvent event = new CoordinatorWriteEvent<>( name, @@ -1498,7 +1560,7 @@ public CompletableFuture scheduleTransactionalWriteOperation( timeout, op ); - enqueue(event); + enqueueLast(event); return event.future; }); } @@ -1537,19 +1599,19 @@ public CompletableFuture scheduleTransactionCompletion( result, timeout ); - enqueue(event); + enqueueLast(event); return event.future; } /** * Schedules a read operation. * - * @param name The name of the write operation. + * @param name The name of the read operation. * @param tp The address of the coordinator (aka its topic-partitions). * @param op The read operation. * * @return A future that will be completed with the result of the read operation - * when the operation is completed or an exception if the write operation failed. + * when the operation is completed or an exception if the read operation failed. * * @param The type of the result. */ @@ -1561,10 +1623,34 @@ public CompletableFuture scheduleReadOperation( throwIfNotRunning(); log.debug("Scheduled execution of read operation {}.", name); CoordinatorReadEvent event = new CoordinatorReadEvent<>(name, tp, op); - enqueue(event); + enqueueLast(event); return event.future; } + /** + * Schedules a read operation for each coordinator. + * + * @param name The name of the read operation. + * @param op The read operation. + * + * @return A list of futures where each future will be completed with the result of the read operation + * when the operation is completed or an exception if the read operation failed. + * + * @param The type of the result. + */ + public List> scheduleReadAllOperation( + String name, + CoordinatorReadOperation op + ) { + throwIfNotRunning(); + log.debug("Scheduled execution of read all operation {}.", name); + return coordinators + .keySet() + .stream() + .map(tp -> scheduleReadOperation(name, tp, op)) + .collect(Collectors.toList()); + } + /** * Schedules an internal event. * @@ -1578,16 +1664,7 @@ private void scheduleInternalOperation( Runnable op ) { log.debug("Scheduled execution of internal operation {}.", name); - enqueue(new CoordinatorInternalEvent(name, tp, op)); - } - - /** - * @return The topic partitions of the coordinators currently registered in the - * runtime. - */ - public Set partitions() { - throwIfNotRunning(); - return new HashSet<>(coordinators.keySet()); + enqueueLast(new CoordinatorInternalEvent(name, tp, op)); } /** @@ -1609,7 +1686,11 @@ public void scheduleLoadOperation( maybeCreateContext(tp); scheduleInternalOperation("Load(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> { - withContextOrThrow(tp, context -> { + // The context is re-created if it does not exist. + CoordinatorContext context = maybeCreateContext(tp); + + context.lock.lock(); + try { if (context.epoch < partitionEpoch) { context.epoch = partitionEpoch; @@ -1617,16 +1698,13 @@ public void scheduleLoadOperation( case FAILED: case INITIAL: context.transitionTo(CoordinatorState.LOADING); - loader.load( - tp, - context.coordinator - ).whenComplete((summary, exception) -> { + loader.load(tp, context.coordinator).whenComplete((summary, exception) -> { scheduleInternalOperation("CompleteLoad(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> { - withContextOrThrow(tp, ctx -> { + CoordinatorContext ctx = coordinators.get(tp); + if (ctx != null) { if (ctx.state != CoordinatorState.LOADING) { - log.info("Ignoring load completion from {} because context is in {} state.", - ctx.tp, ctx.state - ); + log.info("Ignored load completion from {} because context is in {} state.", + ctx.tp, ctx.state); return; } try { @@ -1635,18 +1713,19 @@ public void scheduleLoadOperation( if (summary != null) { runtimeMetrics.recordPartitionLoadSensor(summary.startTimeMs(), summary.endTimeMs()); log.info("Finished loading of metadata from {} with epoch {} in {}ms where {}ms " + - "was spent in the scheduler. Loaded {} records which total to {} bytes.", + "was spent in the scheduler. Loaded {} records which total to {} bytes.", tp, partitionEpoch, summary.endTimeMs() - summary.startTimeMs(), - summary.schedulerQueueTimeMs(), summary.numRecords(), summary.numBytes() - ); + summary.schedulerQueueTimeMs(), summary.numRecords(), summary.numBytes()); } } catch (Throwable ex) { log.error("Failed to load metadata from {} with epoch {} due to {}.", - tp, partitionEpoch, ex.toString(), ex - ); + tp, partitionEpoch, ex.toString()); ctx.transitionTo(CoordinatorState.FAILED); } - }); + } else { + log.debug("Failed to complete the loading of metadata for {} in epoch {} since the coordinator does not exist.", + tp, partitionEpoch); + } }); }); break; @@ -1663,11 +1742,12 @@ public void scheduleLoadOperation( log.error("Cannot load coordinator {} in state {}.", tp, context.state); } } else { - log.info("Ignoring loading metadata from {} since current epoch {} is larger than or equals to {}.", - context.tp, context.epoch, partitionEpoch - ); + log.info("Ignored loading metadata from {} since current epoch {} is larger than or equals to {}.", + context.tp, context.epoch, partitionEpoch); } - }); + } finally { + context.lock.unlock(); + } }); } @@ -1689,8 +1769,8 @@ public void scheduleUnloadOperation( scheduleInternalOperation("UnloadCoordinator(tp=" + tp + ", epoch=" + partitionEpoch + ")", tp, () -> { CoordinatorContext context = coordinators.get(tp); if (context != null) { + context.lock.lock(); try { - context.lock.lock(); if (!partitionEpoch.isPresent() || context.epoch < partitionEpoch.getAsInt()) { log.info("Started unloading metadata for {} with epoch {}.", tp, partitionEpoch); context.transitionTo(CoordinatorState.CLOSED); @@ -1698,16 +1778,14 @@ public void scheduleUnloadOperation( log.info("Finished unloading metadata for {} with epoch {}.", tp, partitionEpoch); } else { log.info("Ignored unloading metadata for {} in epoch {} since current epoch is {}.", - tp, partitionEpoch, context.epoch - ); + tp, partitionEpoch, context.epoch); } } finally { context.lock.unlock(); } } else { log.info("Ignored unloading metadata for {} in epoch {} since metadata was never loaded.", - tp, partitionEpoch - ); + tp, partitionEpoch); } }); } @@ -1731,15 +1809,26 @@ public void onNewMetadataImage( // Push an event for each coordinator. coordinators.keySet().forEach(tp -> { scheduleInternalOperation("UpdateImage(tp=" + tp + ", offset=" + newImage.offset() + ")", tp, () -> { - withContextOrThrow(tp, context -> { - if (context.state == CoordinatorState.ACTIVE) { - log.debug("Applying new metadata image with offset {} to {}.", newImage.offset(), tp); - context.coordinator.onNewMetadataImage(newImage, delta); - } else { - log.debug("Ignoring new metadata image with offset {} for {} because the coordinator is not active.", - newImage.offset(), tp); + CoordinatorContext context = coordinators.get(tp); + if (context != null) { + context.lock.lock(); + try { + if (context.state == CoordinatorState.ACTIVE) { + // The new image can be applied to the coordinator only if the coordinator + // exists and is in the active state. + log.debug("Applying new metadata image with offset {} to {}.", newImage.offset(), tp); + context.coordinator.onNewMetadataImage(newImage, delta); + } else { + log.debug("Ignored new metadata image with offset {} for {} because the coordinator is not active.", + newImage.offset(), tp); + } + } finally { + context.lock.unlock(); } - }); + } else { + log.debug("Ignored new metadata image with offset {} for {} because the coordinator does not exist.", + newImage.offset(), tp); + } }); }); } @@ -1764,7 +1853,12 @@ public void close() throws Exception { Utils.closeQuietly(processor, "event processor"); // Unload all the coordinators. coordinators.forEach((tp, context) -> { - context.transitionTo(CoordinatorState.CLOSED); + context.lock.lock(); + try { + context.transitionTo(CoordinatorState.CLOSED); + } finally { + context.lock.unlock(); + } }); coordinators.clear(); Utils.closeQuietly(runtimeMetrics, "runtime metrics"); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java index f46e8b8a8b..2c22232c47 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/EventAccumulator.java @@ -18,16 +18,15 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.Deque; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Queue; import java.util.Random; import java.util.Set; import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -61,7 +60,7 @@ public interface Event { /** * The map of queues keyed by K. */ - private final Map> queues; + private final Map> queues; /** * The list of available keys. Keys in this list can @@ -111,17 +110,17 @@ public EventAccumulator( } /** - * Adds an {{@link Event}} to the queue. + * Adds an {{@link Event}} at the end of the queue. * * @param event An {{@link Event}}. */ - public void add(T event) throws RejectedExecutionException { + public void addLast(T event) throws RejectedExecutionException { lock.lock(); try { if (closed) throw new RejectedExecutionException("Can't accept an event because the accumulator is closed."); K key = event.key(); - Queue queue = queues.get(key); + Deque queue = queues.get(key); if (queue == null) { queue = new LinkedList<>(); queues.put(key, queue); @@ -129,7 +128,7 @@ public void add(T event) throws RejectedExecutionException { addAvailableKey(key); } } - queue.add(event); + queue.addLast(event); size++; } finally { lock.unlock(); @@ -137,31 +136,69 @@ public void add(T event) throws RejectedExecutionException { } /** - * Returns the next {{@link Event}} available. This method block indefinitely until - * one event is ready or the accumulator is closed. + * Adds an {{@link Event}} at the front of the queue. * - * @return The next event. + * @param event An {{@link Event}}. + */ + public void addFirst(T event) throws RejectedExecutionException { + lock.lock(); + try { + if (closed) throw new RejectedExecutionException("Can't accept an event because the accumulator is closed."); + + K key = event.key(); + Deque queue = queues.get(key); + if (queue == null) { + queue = new LinkedList<>(); + queues.put(key, queue); + if (!inflightKeys.contains(key)) { + addAvailableKey(key); + } + } + queue.addFirst(event); + size++; + } finally { + lock.unlock(); + } + } + + /** + * Returns the next {{@link Event}} available or null if no event is + * available. + * + * @return The next event available or null. */ public T poll() { - return poll(Long.MAX_VALUE, TimeUnit.SECONDS); + lock.lock(); + try { + K key = randomKey(); + if (key == null) return null; + + Deque queue = queues.get(key); + T event = queue.poll(); + + if (queue.isEmpty()) queues.remove(key); + inflightKeys.add(key); + size--; + + return event; + } finally { + lock.unlock(); + } } /** - * Returns the next {{@link Event}} available. This method blocks for the provided - * time and returns null of not event is available. + * Returns the next {{@link Event}} available. This method blocks until an + * event is available or accumulator is closed. * - * @param timeout The timeout. - * @param unit The timeout unit. * @return The next event available or null. */ - public T poll(long timeout, TimeUnit unit) { + public T take() { lock.lock(); try { K key = randomKey(); - long nanos = unit.toNanos(timeout); - while (key == null && !closed && nanos > 0) { + while (key == null && !closed) { try { - nanos = condition.awaitNanos(nanos); + condition.await(); } catch (InterruptedException e) { // Ignore. } @@ -170,7 +207,7 @@ public T poll(long timeout, TimeUnit unit) { if (key == null) return null; - Queue queue = queues.get(key); + Deque queue = queues.get(key); T event = queue.poll(); if (queue.isEmpty()) queues.remove(key); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java index e4adc18e95..0aca6a3b79 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.Objects; import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -128,7 +127,7 @@ private class EventProcessorThread extends Thread { private void handleEvents() { while (!shuttingDown) { recordPollStartTime(time.milliseconds()); - CoordinatorEvent event = accumulator.poll(); + CoordinatorEvent event = accumulator.take(); recordPollEndTime(time.milliseconds()); if (event != null) { try { @@ -148,8 +147,8 @@ private void handleEvents() { } private void drainEvents() { - CoordinatorEvent event = accumulator.poll(0, TimeUnit.MILLISECONDS); - while (event != null) { + CoordinatorEvent event; + while ((event = accumulator.poll()) != null) { try { log.debug("Draining event: {}.", event); metrics.recordEventQueueTime(time.milliseconds() - event.createdTimeMs()); @@ -159,8 +158,6 @@ private void drainEvents() { } finally { accumulator.done(event); } - - event = accumulator.poll(0, TimeUnit.MILLISECONDS); } } @@ -201,14 +198,25 @@ private void recordPollEndTime(long pollEndMs) { } /** - * Enqueues a new {{@link CoordinatorEvent}}. + * Enqueues a new {{@link CoordinatorEvent}} at the end of the processor. + * + * @param event The event. + * @throws RejectedExecutionException If the event processor is closed. + */ + @Override + public void enqueueLast(CoordinatorEvent event) throws RejectedExecutionException { + accumulator.addLast(event); + } + + /** + * Enqueues a new {{@link CoordinatorEvent}} at the front of the processor. * * @param event The event. * @throws RejectedExecutionException If the event processor is closed. */ @Override - public void enqueue(CoordinatorEvent event) throws RejectedExecutionException { - accumulator.add(event); + public void enqueueFirst(CoordinatorEvent event) throws RejectedExecutionException { + accumulator.addFirst(event); } /** diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/PartitionWriter.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/PartitionWriter.java index d647fce149..6ca2d2733f 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/PartitionWriter.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/PartitionWriter.java @@ -128,6 +128,7 @@ long appendEndTransactionMarker( * @param transactionalId The transactional id. * @param producerId The producer id. * @param producerEpoch The producer epoch. + * @param apiVersion The version of the Request used. * @return A future failed with any error encountered; or the {@link VerificationGuard} * if the transaction required verification and {@link VerificationGuard#SENTINEL} * if it did not. @@ -137,6 +138,7 @@ CompletableFuture maybeStartTransactionVerification( TopicPartition tp, String transactionalId, long producerId, - short producerEpoch + short producerEpoch, + short apiVersion ) throws KafkaException; } diff --git a/group-coordinator/src/main/resources/common/message/ConsumerGroupCurrentMemberAssignmentValue.json b/group-coordinator/src/main/resources/common/message/ConsumerGroupCurrentMemberAssignmentValue.json index 575f8ac133..b91665e69d 100644 --- a/group-coordinator/src/main/resources/common/message/ConsumerGroupCurrentMemberAssignmentValue.json +++ b/group-coordinator/src/main/resources/common/message/ConsumerGroupCurrentMemberAssignmentValue.json @@ -24,20 +24,12 @@ "about": "The current member epoch that is expected from the member in the heartbeat request." }, { "name": "PreviousMemberEpoch", "versions": "0+", "type": "int32", "about": "If the last epoch bump is lost before reaching the member, the member will retry with the previous epoch." }, - { "name": "TargetMemberEpoch", "versions": "0+", "type": "int32", - "about": "The target epoch corresponding to the assignment used to compute the AssignedPartitions, the PartitionsPendingRevocation and the PartitionsPendingAssignment fields." }, + { "name": "State", "versions": "0+", "type": "int8", + "about": "The member state. See ConsumerGroupMember.MemberState for the possible values." }, { "name": "AssignedPartitions", "versions": "0+", "type": "[]TopicPartitions", "about": "The partitions assigned to (or owned by) this member." }, { "name": "PartitionsPendingRevocation", "versions": "0+", "type": "[]TopicPartitions", - "about": "The partitions that must be revoked by this member." }, - { "name": "PartitionsPendingAssignment", "versions": "0+", "type": "[]TopicPartitions", - "about": "The partitions that will be assigned to this member when they are freed up by their current owners." }, - { "name": "Error", "versions": "0+", "type": "int8", - "about": "The error reported by the assignor." }, - { "name": "MetadataVersion", "versions": "0+", "type": "int16", - "about": "The version of the metadata bytes." }, - { "name": "MetadataBytes", "versions": "0+", "type": "bytes", - "about": "The metadata bytes." } + "about": "The partitions that must be revoked by this member." } ], "commonStructs": [ { "name": "TopicPartitions", "versions": "0+", "fields": [ diff --git a/group-coordinator/src/main/resources/common/message/ConsumerGroupMemberMetadataValue.json b/group-coordinator/src/main/resources/common/message/ConsumerGroupMemberMetadataValue.json index 3b83f95985..4886b7ad9d 100644 --- a/group-coordinator/src/main/resources/common/message/ConsumerGroupMemberMetadataValue.json +++ b/group-coordinator/src/main/resources/common/message/ConsumerGroupMemberMetadataValue.json @@ -35,21 +35,6 @@ { "name": "RebalanceTimeoutMs", "type": "int32", "versions": "0+", "default": -1, "about": "The rebalance timeout" }, { "name": "ServerAssignor", "versions": "0+", "nullableVersions": "0+", "type": "string", - "about": "The server assignor to use; or null if not used." }, - { "name": "Assignors", "versions": "0+", "type": "[]Assignor", - "about": "The list of assignors.", "fields": [ - { "name": "Name", "versions": "0+", "type": "string", - "about": "The assignor name." }, - { "name": "MinimumVersion", "versions": "0+", "type": "int16", - "about": "The minimum version supported by the assignor." }, - { "name": "MaximumVersion", "versions": "0+", "type": "int16", - "about": "The maximum version supported by the assignor." }, - { "name": "Reason", "versions": "0+", "type": "int8", - "about": "The reason reported by the assignor." }, - { "name": "Version", "versions": "0+", "type": "int16", - "about": "The version used to serialize the metadata." }, - { "name": "Metadata", "versions": "0+", "type": "bytes", - "about": "The metadata." } - ]} + "about": "The server assignor to use; or null if not used." } ] } diff --git a/group-coordinator/src/main/resources/common/message/ConsumerGroupTargetAssignmentMemberValue.json b/group-coordinator/src/main/resources/common/message/ConsumerGroupTargetAssignmentMemberValue.json index d056eab6d8..e05c28928e 100644 --- a/group-coordinator/src/main/resources/common/message/ConsumerGroupTargetAssignmentMemberValue.json +++ b/group-coordinator/src/main/resources/common/message/ConsumerGroupTargetAssignmentMemberValue.json @@ -20,16 +20,10 @@ "validVersions": "0", "flexibleVersions": "0+", "fields": [ - { "name": "Error", "versions": "0+", "type": "int8", - "about": "The assigned error."}, { "name": "TopicPartitions", "versions": "0+", "type": "[]TopicPartition", "about": "The assigned partitions.", "fields": [ { "name": "TopicId", "versions": "0+", "type": "uuid" }, { "name": "Partitions", "versions": "0+", "type": "[]int32" } - ]}, - { "name": "MetadataVersion", "versions": "0+", "type": "int16", - "about": "The version of the assigned metadata." }, - { "name": "MetadataBytes", "versions": "0+", "type": "bytes", - "about": "The assigned metadata." } + ]} ] } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/Assertions.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/Assertions.java index 77d4c7d03c..b5f40240eb 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/Assertions.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/Assertions.java @@ -144,18 +144,12 @@ private static void assertApiMessageAndVersionEquals( assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch()); assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch()); - assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch()); - assertEquals(expectedValue.error(), actualValue.error()); - assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion()); - assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes()); // We transform those to Maps before comparing them. assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()), fromTopicPartitions(actualValue.assignedPartitions())); assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()), fromTopicPartitions(actualValue.partitionsPendingRevocation())); - assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()), - fromTopicPartitions(actualValue.partitionsPendingAssignment())); } else if (actual.message() instanceof ConsumerGroupPartitionMetadataValue) { // The order of the racks stored in the PartitionMetadata of the ConsumerGroupPartitionMetadataValue // is not always guaranteed. Therefore, we need a special comparator. diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 2404d304be..ee2defb5ae 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -73,11 +73,11 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.NullSource; import org.junit.jupiter.params.provider.ValueSource; import org.mockito.ArgumentMatchers; -import org.mockito.internal.util.collections.Sets; import java.net.InetAddress; import java.time.Duration; @@ -91,8 +91,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; -import java.util.stream.IntStream; import java.util.stream.Stream; import static org.apache.kafka.common.requests.JoinGroupRequest.UNKNOWN_MEMBER_ID; @@ -759,10 +757,7 @@ public void testListGroups() throws ExecutionException, InterruptedException, Ti runtime, new GroupCoordinatorMetrics() ); - int partitionCount = 3; - service.startup(() -> partitionCount); - - ListGroupsRequestData request = new ListGroupsRequestData(); + service.startup(() -> 3); List expectedResults = Arrays.asList( new ListGroupsResponseData.ListedGroup() @@ -781,26 +776,22 @@ public void testListGroups() throws ExecutionException, InterruptedException, Ti .setGroupState("Dead") .setGroupType("consumer") ); - when(runtime.partitions()).thenReturn(Sets.newSet( - new TopicPartition("__consumer_offsets", 0), - new TopicPartition("__consumer_offsets", 1), - new TopicPartition("__consumer_offsets", 2) + + when(runtime.scheduleReadAllOperation( + ArgumentMatchers.eq("list-groups"), + ArgumentMatchers.any() + )).thenReturn(Arrays.asList( + CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(0))), + CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(1))), + CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(2))) )); - for (int i = 0; i < partitionCount; i++) { - when(runtime.scheduleReadOperation( - ArgumentMatchers.eq("list-groups"), - ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", i)), - ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(i)))); - } CompletableFuture responseFuture = service.listGroups( requestContext(ApiKeys.LIST_GROUPS), - request + new ListGroupsRequestData() ); - List actualResults = responseFuture.get(5, TimeUnit.SECONDS).groups(); - assertEquals(expectedResults, actualResults); + assertEquals(expectedResults, responseFuture.get(5, TimeUnit.SECONDS).groups()); } @Test @@ -813,8 +804,7 @@ public void testListGroupsFailedWithNotCoordinatorException() runtime, new GroupCoordinatorMetrics() ); - int partitionCount = 3; - service.startup(() -> partitionCount); + service.startup(() -> 3); List expectedResults = Arrays.asList( new ListGroupsResponseData.ListedGroup() @@ -829,36 +819,25 @@ public void testListGroupsFailedWithNotCoordinatorException() .setGroupType("consumer") ); - ListGroupsRequestData request = new ListGroupsRequestData(); - when(runtime.partitions()).thenReturn(Sets.newSet( - new TopicPartition("__consumer_offsets", 0), - new TopicPartition("__consumer_offsets", 1), - new TopicPartition("__consumer_offsets", 2) - )); - for (int i = 0; i < 2; i++) { - when(runtime.scheduleReadOperation( - ArgumentMatchers.eq("list-groups"), - ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", i)), - ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(i)))); - } - - when(runtime.scheduleReadOperation( + when(runtime.scheduleReadAllOperation( ArgumentMatchers.eq("list-groups"), - ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 2)), ArgumentMatchers.any() - )).thenReturn(FutureUtils.failedFuture(new NotCoordinatorException(""))); + )).thenReturn(Arrays.asList( + CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(0))), + CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(1))), + FutureUtils.failedFuture(new NotCoordinatorException("")) + )); CompletableFuture responseFuture = service.listGroups( requestContext(ApiKeys.LIST_GROUPS), - request + new ListGroupsRequestData() ); - List actualResults = responseFuture.get(5, TimeUnit.SECONDS).groups(); - assertEquals(expectedResults, actualResults); + + assertEquals(expectedResults, responseFuture.get(5, TimeUnit.SECONDS).groups()); } @Test - public void testListGroupsFailedImmediately() + public void testListGroupsWithFailure() throws InterruptedException, ExecutionException, TimeoutException { CoordinatorRuntime runtime = mockRuntime(); GroupCoordinatorService service = new GroupCoordinatorService( @@ -867,37 +846,27 @@ public void testListGroupsFailedImmediately() runtime, new GroupCoordinatorMetrics() ); - int partitionCount = 3; - service.startup(() -> partitionCount); - - ListGroupsRequestData request = new ListGroupsRequestData(); - when(runtime.partitions()).thenReturn(Sets.newSet( - new TopicPartition("__consumer_offsets", 0), - new TopicPartition("__consumer_offsets", 1), - new TopicPartition("__consumer_offsets", 2) - )); - for (int i = 0; i < 2; i++) { - when(runtime.scheduleReadOperation( - ArgumentMatchers.eq("list-groups"), - ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", i)), - ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.emptyList())); - } + service.startup(() -> 3); - when(runtime.scheduleReadOperation( + when(runtime.scheduleReadAllOperation( ArgumentMatchers.eq("list-groups"), - ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 2)), ArgumentMatchers.any() - )).thenReturn(FutureUtils.failedFuture(new CoordinatorLoadInProgressException(""))); + )).thenReturn(Arrays.asList( + CompletableFuture.completedFuture(Collections.emptyList()), + CompletableFuture.completedFuture(Collections.emptyList()), + FutureUtils.failedFuture(new CoordinatorLoadInProgressException("")) + )); CompletableFuture responseFuture = service.listGroups( requestContext(ApiKeys.LIST_GROUPS), - request + new ListGroupsRequestData() ); - ListGroupsResponseData listGroupsResponseData = responseFuture.get(5, TimeUnit.SECONDS); - assertEquals(Errors.COORDINATOR_LOAD_IN_PROGRESS.code(), listGroupsResponseData.errorCode()); - assertEquals(Collections.emptyList(), listGroupsResponseData.groups()); + assertEquals( + new ListGroupsResponseData() + .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()), + responseFuture.get(5, TimeUnit.SECONDS) + ); } @Test @@ -1703,12 +1672,6 @@ public void testDeleteGroups() throws Exception { result1.duplicate() )); - when(runtime.partitions()).thenReturn(Sets.newSet( - new TopicPartition("__consumer_offsets", 0), - new TopicPartition("__consumer_offsets", 1), - new TopicPartition("__consumer_offsets", 2) - )); - when(runtime.scheduleWriteOperation( ArgumentMatchers.eq("delete-groups"), ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 2)), @@ -1925,6 +1888,7 @@ public void testCommitTransactionalOffsets() throws ExecutionException, Interrup ArgumentMatchers.eq(10L), ArgumentMatchers.eq((short) 5), ArgumentMatchers.eq(Duration.ofMillis(5000)), + ArgumentMatchers.any(), ArgumentMatchers.any() )).thenReturn(CompletableFuture.completedFuture(response)); @@ -1937,8 +1901,15 @@ public void testCommitTransactionalOffsets() throws ExecutionException, Interrup assertEquals(response, future.get()); } - @Test - public void testCommitTransactionalOffsetsWithWrappedError() throws ExecutionException, InterruptedException { + @ParameterizedTest + @CsvSource({ + "NOT_ENOUGH_REPLICAS, COORDINATOR_NOT_AVAILABLE", + "NETWORK_EXCEPTION, COORDINATOR_LOAD_IN_PROGRESS" + }) + public void testCommitTransactionalOffsetsWithWrappedError( + Errors error, + Errors expectedError + ) throws ExecutionException, InterruptedException { CoordinatorRuntime runtime = mockRuntime(); GroupCoordinatorService service = new GroupCoordinatorService( new LogContext(), @@ -1966,7 +1937,7 @@ public void testCommitTransactionalOffsetsWithWrappedError() throws ExecutionExc .setName("topic") .setPartitions(Collections.singletonList(new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(0) - .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()))))); + .setErrorCode(expectedError.code()))))); when(runtime.scheduleTransactionalWriteOperation( ArgumentMatchers.eq("txn-commit-offset"), @@ -1975,8 +1946,9 @@ public void testCommitTransactionalOffsetsWithWrappedError() throws ExecutionExc ArgumentMatchers.eq(10L), ArgumentMatchers.eq((short) 5), ArgumentMatchers.eq(Duration.ofMillis(5000)), + ArgumentMatchers.any(), ArgumentMatchers.any() - )).thenReturn(FutureUtils.failedFuture(new CompletionException(Errors.NOT_ENOUGH_REPLICAS.exception()))); + )).thenReturn(FutureUtils.failedFuture(new CompletionException(error.exception()))); CompletableFuture future = service.commitTransactionalOffsets( requestContext(ApiKeys.TXN_OFFSET_COMMIT), @@ -2067,7 +2039,6 @@ public void testCompleteTransactionWithUnexpectedPartition() { @Test public void testOnPartitionsDeleted() { - int partitionCount = 3; CoordinatorRuntime runtime = mockRuntime(); GroupCoordinatorService service = new GroupCoordinatorService( new LogContext(), @@ -2075,36 +2046,17 @@ public void testOnPartitionsDeleted() { runtime, new GroupCoordinatorMetrics() ); + service.startup(() -> 3); - service.startup(() -> partitionCount); - - when(runtime.partitions()).thenReturn( - IntStream - .range(0, partitionCount) - .mapToObj(i -> new TopicPartition("__consumer_offsets", i)) - .collect(Collectors.toSet()) - ); - - List> futures = IntStream - .range(0, partitionCount) - .mapToObj(__ -> new CompletableFuture()) - .collect(Collectors.toList()); - - IntStream.range(0, partitionCount).forEach(i -> { - CompletableFuture future = futures.get(i); - when(runtime.scheduleWriteOperation( - ArgumentMatchers.eq("on-partition-deleted"), - ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", i)), - ArgumentMatchers.eq(Duration.ofMillis(5000)), - ArgumentMatchers.any() - )).thenAnswer(__ -> future); - }); - - IntStream.range(0, partitionCount - 1).forEach(i -> { - futures.get(i).complete(null); - }); - - futures.get(partitionCount - 1).completeExceptionally(Errors.COORDINATOR_LOAD_IN_PROGRESS.exception()); + when(runtime.scheduleWriteAllOperation( + ArgumentMatchers.eq("on-partition-deleted"), + ArgumentMatchers.eq(Duration.ofMillis(5000)), + ArgumentMatchers.any() + )).thenReturn(Arrays.asList( + CompletableFuture.completedFuture(null), + CompletableFuture.completedFuture(null), + FutureUtils.failedFuture(Errors.COORDINATOR_LOAD_IN_PROGRESS.exception()) + )); // The exception is logged and swallowed. assertDoesNotThrow(() -> diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java index 1e6930a802..59868f36f1 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java @@ -225,14 +225,14 @@ public void testDeleteGroups() { List records = invocation.getArgument(1); records.add(RecordHelpers.newGroupMetadataTombstoneRecord(groupId)); return null; - }).when(groupMetadataManager).deleteGroup(anyString(), anyList()); + }).when(groupMetadataManager).createGroupTombstoneRecords(anyString(), anyList()); CoordinatorResult coordinatorResult = coordinator.deleteGroups(context, groupIds); for (String groupId : groupIds) { verify(groupMetadataManager, times(1)).validateDeleteGroup(ArgumentMatchers.eq(groupId)); - verify(groupMetadataManager, times(1)).deleteGroup(ArgumentMatchers.eq(groupId), anyList()); + verify(groupMetadataManager, times(1)).createGroupTombstoneRecords(ArgumentMatchers.eq(groupId), anyList()); verify(offsetMetadataManager, times(1)).deleteAllOffsets(ArgumentMatchers.eq(groupId), anyList()); } assertEquals(expectedResult, coordinatorResult); @@ -291,7 +291,7 @@ public void testDeleteGroupsInvalidGroupId() { List records = invocation.getArgument(1); records.add(RecordHelpers.newGroupMetadataTombstoneRecord(groupId)); return null; - }).when(groupMetadataManager).deleteGroup(anyString(), anyList()); + }).when(groupMetadataManager).createGroupTombstoneRecords(anyString(), anyList()); CoordinatorResult coordinatorResult = coordinator.deleteGroups(context, groupIds); @@ -299,7 +299,7 @@ public void testDeleteGroupsInvalidGroupId() { for (String groupId : groupIds) { verify(groupMetadataManager, times(1)).validateDeleteGroup(eq(groupId)); if (!groupId.equals("group-id-2")) { - verify(groupMetadataManager, times(1)).deleteGroup(eq(groupId), anyList()); + verify(groupMetadataManager, times(1)).createGroupTombstoneRecords(eq(groupId), anyList()); verify(offsetMetadataManager, times(1)).deleteAllOffsets(eq(groupId), anyList()); } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 4a47fd3dc3..dc21a2140d 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -65,6 +65,7 @@ import org.apache.kafka.coordinator.group.consumer.ConsumerGroup; import org.apache.kafka.coordinator.group.consumer.ConsumerGroupBuilder; import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember; +import org.apache.kafka.coordinator.group.consumer.MemberState; import org.apache.kafka.coordinator.group.consumer.TopicMetadata; import org.apache.kafka.coordinator.group.generated.GroupMetadataValue; import org.apache.kafka.coordinator.group.classic.ClassicGroup; @@ -106,7 +107,7 @@ import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; import static org.apache.kafka.coordinator.group.GroupMetadataManager.appendGroupMetadataErrorToResponseError; -import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupRevocationTimeoutKey; +import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupRebalanceTimeoutKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupSessionTimeoutKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.EMPTY_RESULT; import static org.apache.kafka.coordinator.group.GroupMetadataManager.classicGroupHeartbeatKey; @@ -134,7 +135,6 @@ import static org.mockito.Mockito.when; public class GroupMetadataManagerTest { - @Test public void testConsumerHeartbeatRequestValidation() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); @@ -336,9 +336,9 @@ public void testConsumerGroupMemberEpochValidation() { .build(); ConsumerGroupMember member = new ConsumerGroupMember.Builder(memberId) + .setState(MemberState.STABLE) .setMemberEpoch(100) .setPreviousMemberEpoch(99) - .setTargetMemberEpoch(100) .setRebalanceTimeoutMs(5000) .setClientId("client") .setClientHost("localhost/127.0.0.1") @@ -434,7 +434,7 @@ public void testMemberJoinsEmptyConsumerGroup() { )); assertThrows(GroupIdNotFoundException.class, () -> - context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false)); + context.groupMetadataManager.consumerGroup(groupId)); CoordinatorResult result = context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() @@ -464,9 +464,9 @@ public void testMemberJoinsEmptyConsumerGroup() { ); ConsumerGroupMember expectedMember = new ConsumerGroupMember.Builder(memberId) + .setState(MemberState.STABLE) .setMemberEpoch(1) .setPreviousMemberEpoch(0) - .setTargetMemberEpoch(1) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -516,9 +516,9 @@ public void testUpdatingSubscriptionTriggersNewTargetAssignment() { .build()) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId) + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Collections.singletonList("foo")) @@ -563,9 +563,9 @@ public void testUpdatingSubscriptionTriggersNewTargetAssignment() { ); ConsumerGroupMember expectedMember = new ConsumerGroupMember.Builder(memberId) + .setState(MemberState.STABLE) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(11) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -618,9 +618,9 @@ public void testNewJoiningMemberTriggersNewTargetAssignment() { .build()) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -631,9 +631,9 @@ public void testNewJoiningMemberTriggersNewTargetAssignment() { mkTopicAssignment(barTopicId, 0, 1))) .build()) .withMember(new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -692,17 +692,14 @@ public void testNewJoiningMemberTriggersNewTargetAssignment() { ); ConsumerGroupMember expectedMember3 = new ConsumerGroupMember.Builder(memberId3) + .setState(MemberState.UNRELEASED_PARTITIONS) .setMemberEpoch(11) .setPreviousMemberEpoch(0) - .setTargetMemberEpoch(11) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(Arrays.asList("foo", "bar")) .setServerAssignorName("range") - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 4, 5), - mkTopicAssignment(barTopicId, 2))) .build(); List expectedRecords = Arrays.asList( @@ -762,9 +759,9 @@ public void testLeavingMemberBumpsGroupEpoch() { .build()) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -774,9 +771,9 @@ public void testLeavingMemberBumpsGroupEpoch() { mkTopicAssignment(barTopicId, 0, 1))) .build()) .withMember(new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") // Use zar only here to ensure that metadata needs to be recomputed. @@ -854,10 +851,10 @@ public void testGroupEpochBumpWhenNewStaticMemberJoins() { .build()) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setInstanceId(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -867,10 +864,10 @@ public void testGroupEpochBumpWhenNewStaticMemberJoins() { mkTopicAssignment(barTopicId, 0, 1))) .build()) .withMember(new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.STABLE) .setInstanceId(memberId2) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") // Use zar only here to ensure that metadata needs to be recomputed. @@ -931,17 +928,14 @@ public void testGroupEpochBumpWhenNewStaticMemberJoins() { ConsumerGroupMember expectedMember3 = new ConsumerGroupMember.Builder(memberId3) .setMemberEpoch(11) + .setState(MemberState.UNRELEASED_PARTITIONS) .setInstanceId(memberId3) .setPreviousMemberEpoch(0) - .setTargetMemberEpoch(11) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(Arrays.asList("foo", "bar")) .setServerAssignorName("range") - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 4, 5), - mkTopicAssignment(barTopicId, 2))) .build(); List expectedRecords = Arrays.asList( @@ -989,10 +983,10 @@ public void testStaticMemberGetsBackAssignmentUponRejoin() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); ConsumerGroupMember member1 = new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setInstanceId(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -1002,10 +996,10 @@ public void testStaticMemberGetsBackAssignmentUponRejoin() { mkTopicAssignment(barTopicId, 0, 1))) .build(); ConsumerGroupMember member2 = new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.STABLE) .setInstanceId(memberId2) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setRebalanceTimeoutMs(5000) .setClientId("client") .setClientHost("localhost/127.0.0.1") @@ -1114,10 +1108,10 @@ public void testStaticMemberGetsBackAssignmentUponRejoin() { ); ConsumerGroupMember expectedRejoinedMember = new ConsumerGroupMember.Builder(member2RejoinId) + .setState(MemberState.STABLE) .setMemberEpoch(10) .setInstanceId(memberId2) .setPreviousMemberEpoch(0) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -1143,7 +1137,7 @@ public void testStaticMemberGetsBackAssignmentUponRejoin() { assertRecordsEquals(expectedRecordsAfterRejoin, rejoinResult.records()); // Verify that there are no timers. context.assertNoSessionTimeout(groupId, memberId2); - context.assertNoRevocationTimeout(groupId, memberId2); + context.assertNoRebalanceTimeout(groupId, memberId2); } @Test @@ -1160,10 +1154,10 @@ public void testNoGroupEpochBumpWhenStaticMemberTemporarilyLeaves() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); ConsumerGroupMember member1 = new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setInstanceId(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -1173,10 +1167,10 @@ public void testNoGroupEpochBumpWhenStaticMemberTemporarilyLeaves() { mkTopicAssignment(barTopicId, 0, 1))) .build(); ConsumerGroupMember member2 = new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.STABLE) .setInstanceId(memberId2) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") // Use zar only here to ensure that metadata needs to be recomputed. @@ -1262,10 +1256,10 @@ public void testLeavingStaticMemberBumpsGroupEpoch() { .build()) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setInstanceId(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -1275,10 +1269,10 @@ public void testLeavingStaticMemberBumpsGroupEpoch() { mkTopicAssignment(barTopicId, 0, 1))) .build()) .withMember(new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.STABLE) .setInstanceId(memberId2) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") // Use zar only here to ensure that metadata needs to be recomputed. @@ -1353,10 +1347,10 @@ public void testShouldThrownUnreleasedInstanceIdExceptionWhenNewMemberJoinsWithI .build()) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setInstanceId(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -1402,10 +1396,10 @@ public void testShouldThrownUnknownMemberIdExceptionWhenUnknownStaticMemberJoins .build()) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setInstanceId(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -1450,10 +1444,10 @@ public void testShouldThrowFencedInstanceIdExceptionWhenStaticMemberWithDifferen .build()) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setInstanceId(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -1490,10 +1484,10 @@ public void testConsumerGroupMemberEpochValidationForStaticMember() { .build(); ConsumerGroupMember member = new ConsumerGroupMember.Builder(memberId) + .setState(MemberState.STABLE) .setInstanceId(memberId) .setMemberEpoch(100) .setPreviousMemberEpoch(99) - .setTargetMemberEpoch(100) .setRebalanceTimeoutMs(5000) .setClientId("client") .setClientHost("localhost/127.0.0.1") @@ -1583,10 +1577,10 @@ public void testShouldThrowUnknownMemberIdExceptionWhenUnknownStaticMemberLeaves .build()) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setInstanceId(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -1629,10 +1623,10 @@ public void testShouldThrowFencedInstanceIdExceptionWhenStaticMemberWithDifferen .build()) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setInstanceId(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -1656,6 +1650,102 @@ public void testShouldThrowFencedInstanceIdExceptionWhenStaticMemberWithDifferen .setTopicPartitions(Collections.emptyList()))); } + @Test + public void testConsumerGroupHeartbeatFullResponse() { + String groupId = "fooup"; + String memberId = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + + // Create a context with an empty consumer group. + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 2) + .addRacks() + .build()) + .build(); + + // Prepare new assignment for the group. + assignor.prepareGroupAssignment(new GroupAssignment( + new HashMap() { + { + put(memberId, new MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1) + ))); + } + } + )); + + CoordinatorResult result; + + // A full response should be sent back on joining. + result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignor("range") + .setTopicPartitions(Collections.emptyList())); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(Collections.singletonList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(0, 1))))), + result.response() + ); + + // Otherwise, a partial response should be sent back. + result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(result.response().memberEpoch())); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000), + result.response() + ); + + // A full response should be sent back when the member sends + // a full request again. + result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(result.response().memberEpoch()) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignor("range") + .setTopicPartitions(Collections.emptyList())); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(Collections.singletonList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(0, 1))))), + result.response() + ); + } + @Test public void testReconciliationProcess() { String groupId = "fooup"; @@ -1680,9 +1770,9 @@ public void testReconciliationProcess() { .build()) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -1693,9 +1783,9 @@ public void testReconciliationProcess() { mkTopicAssignment(barTopicId, 0, 1))) .build()) .withMember(new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -1737,13 +1827,14 @@ public void testReconciliationProcess() { CoordinatorResult result; // Members in the group are in Stable state. - assertEquals(ConsumerGroupMember.MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId1)); - assertEquals(ConsumerGroupMember.MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId2)); + assertEquals(MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId1)); + assertEquals(MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId2)); assertEquals(ConsumerGroup.ConsumerGroupState.STABLE, context.consumerGroupState(groupId)); // Member 3 joins the group. This triggers the computation of a new target assignment // for the group. Member 3 does not get any assigned partitions yet because they are - // all owned by other members. However, it transitions to epoch 11 / Assigning state. + // all owned by other members. However, it transitions to epoch 11 and the + // Unreleased Partitions state. result = context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() .setGroupId(groupId) @@ -1767,21 +1858,18 @@ public void testReconciliationProcess() { // already covered by other tests. assertRecordEquals( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) + .setState(MemberState.UNRELEASED_PARTITIONS) .setMemberEpoch(11) .setPreviousMemberEpoch(0) - .setTargetMemberEpoch(11) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 4, 5), - mkTopicAssignment(barTopicId, 1))) .build()), result.records().get(result.records().size() - 1) ); - assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId3)); + assertEquals(MemberState.UNRELEASED_PARTITIONS, context.consumerGroupMemberState(groupId, memberId3)); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); - // Member 1 heartbeats. It remains at epoch 10 but transitions to Revoking state until - // it acknowledges the revocation of its partitions. The response contains the new + // Member 1 heartbeats. It remains at epoch 10 but transitions to Unrevoked Partitions + // state until it acknowledges the revocation of its partitions. The response contains the new // assignment without the partitions that must be revoked. result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() .setGroupId(groupId) @@ -1807,9 +1895,9 @@ public void testReconciliationProcess() { assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.UNREVOKED_PARTITIONS) .setMemberEpoch(10) - .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(11) + .setPreviousMemberEpoch(10) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 0, 1), mkTopicAssignment(barTopicId, 0))) @@ -1820,11 +1908,11 @@ public void testReconciliationProcess() { result.records() ); - assertEquals(ConsumerGroupMember.MemberState.REVOKING, context.consumerGroupMemberState(groupId, memberId1)); + assertEquals(MemberState.UNREVOKED_PARTITIONS, context.consumerGroupMemberState(groupId, memberId1)); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); - // Member 2 heartbeats. It remains at epoch 10 but transitions to Revoking state until - // it acknowledges the revocation of its partitions. The response contains the new + // Member 2 heartbeats. It remains at epoch 10 but transitions to Unrevoked Partitions + // state until it acknowledges the revocation of its partitions. The response contains the new // assignment without the partitions that must be revoked. result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() .setGroupId(groupId) @@ -1850,21 +1938,19 @@ public void testReconciliationProcess() { assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.UNREVOKED_PARTITIONS) .setMemberEpoch(10) - .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(11) + .setPreviousMemberEpoch(10) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 3), mkTopicAssignment(barTopicId, 2))) .setPartitionsPendingRevocation(mkAssignment( mkTopicAssignment(fooTopicId, 4, 5))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 2))) .build())), result.records() ); - assertEquals(ConsumerGroupMember.MemberState.REVOKING, context.consumerGroupMemberState(groupId, memberId2)); + assertEquals(MemberState.UNREVOKED_PARTITIONS, context.consumerGroupMemberState(groupId, memberId2)); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); // Member 3 heartbeats. The response does not contain any assignment @@ -1882,8 +1968,16 @@ public void testReconciliationProcess() { result.response() ); - assertEquals(Collections.emptyList(), result.records()); - assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId3)); + assertRecordsEquals(Collections.singletonList( + RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) + .setState(MemberState.UNRELEASED_PARTITIONS) + .setMemberEpoch(11) + .setPreviousMemberEpoch(11) + .build())), + result.records() + ); + + assertEquals(MemberState.UNRELEASED_PARTITIONS, context.consumerGroupMemberState(groupId, memberId3)); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); // Member 1 acknowledges the revocation of the partitions. It does so by providing the @@ -1906,24 +2000,15 @@ public void testReconciliationProcess() { new ConsumerGroupHeartbeatResponseData() .setMemberId(memberId1) .setMemberEpoch(11) - .setHeartbeatIntervalMs(5000) - .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Arrays.asList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(fooTopicId) - .setPartitions(Arrays.asList(0, 1)), - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(barTopicId) - .setPartitions(Collections.singletonList(0)) - ))), + .setHeartbeatIntervalMs(5000), result.response() ); assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 0, 1), mkTopicAssignment(barTopicId, 0))) @@ -1931,7 +2016,7 @@ public void testReconciliationProcess() { result.records() ); - assertEquals(ConsumerGroupMember.MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId1)); + assertEquals(MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId1)); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); // Member 2 heartbeats but without acknowledging the revocation yet. This is basically a no-op. @@ -1949,11 +2034,11 @@ public void testReconciliationProcess() { ); assertEquals(Collections.emptyList(), result.records()); - assertEquals(ConsumerGroupMember.MemberState.REVOKING, context.consumerGroupMemberState(groupId, memberId2)); + assertEquals(MemberState.UNREVOKED_PARTITIONS, context.consumerGroupMemberState(groupId, memberId2)); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); // Member 3 heartbeats. It receives the partitions revoked by member 1 but remains - // in Assigning state because it still waits on other partitions. + // in Unreleased Partitions state because it still waits on other partitions. result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() .setGroupId(groupId) .setMemberId(memberId3) @@ -1974,18 +2059,16 @@ public void testReconciliationProcess() { assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) + .setState(MemberState.UNRELEASED_PARTITIONS) .setMemberEpoch(11) .setPreviousMemberEpoch(11) - .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(barTopicId, 1))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 4, 5))) .build())), result.records() ); - assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId3)); + assertEquals(MemberState.UNRELEASED_PARTITIONS, context.consumerGroupMemberState(groupId, memberId3)); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); // Member 3 heartbeats. Member 2 has not acknowledged the revocation of its partition so @@ -2004,7 +2087,7 @@ public void testReconciliationProcess() { ); assertEquals(Collections.emptyList(), result.records()); - assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId3)); + assertEquals(MemberState.UNRELEASED_PARTITIONS, context.consumerGroupMemberState(groupId, memberId3)); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); // Member 2 acknowledges the revocation of the partitions. It does so by providing the @@ -2042,9 +2125,9 @@ public void testReconciliationProcess() { assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.STABLE) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 2, 3), mkTopicAssignment(barTopicId, 2))) @@ -2052,14 +2135,19 @@ public void testReconciliationProcess() { result.records() ); - assertEquals(ConsumerGroupMember.MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId2)); + assertEquals(MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId2)); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); - // Member 3 heartbeats. It receives all its partitions and transitions to Stable. + // Member 3 heartbeats to acknowledge its current assignment. It receives all its partitions and + // transitions to Stable state. result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() .setGroupId(groupId) .setMemberId(memberId3) - .setMemberEpoch(11)); + .setMemberEpoch(11) + .setTopicPartitions(Collections.singletonList( + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Collections.singletonList(1))))); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -2079,9 +2167,9 @@ public void testReconciliationProcess() { assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) + .setState(MemberState.STABLE) .setMemberEpoch(11) .setPreviousMemberEpoch(11) - .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 4, 5), mkTopicAssignment(barTopicId, 1))) @@ -2089,243 +2177,10 @@ public void testReconciliationProcess() { result.records() ); - assertEquals(ConsumerGroupMember.MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId3)); + assertEquals(MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId3)); assertEquals(ConsumerGroup.ConsumerGroupState.STABLE, context.consumerGroupState(groupId)); } - @Test - public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { - String groupId = "fooup"; - // Use a static member id as it makes the test easier. - String memberId1 = Uuid.randomUuid().toString(); - String memberId2 = Uuid.randomUuid().toString(); - String memberId3 = Uuid.randomUuid().toString(); - - Uuid fooTopicId = Uuid.randomUuid(); - String fooTopicName = "foo"; - - // Create a context with one consumer group containing one member. - MockPartitionAssignor assignor = new MockPartitionAssignor("range"); - GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) - .withMetadataImage(new MetadataImageBuilder() - .addTopic(fooTopicId, fooTopicName, 6) - .addRacks() - .build()) - .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) - .withMember(new ConsumerGroupMember.Builder(memberId1) - .setMemberEpoch(10) - .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) - .setClientId("client") - .setClientHost("localhost/127.0.0.1") - .setRebalanceTimeoutMs(5000) - .setSubscribedTopicNames(Arrays.asList("foo", "bar")) - .setServerAssignorName("range") - .setAssignedPartitions(mkAssignment( - mkTopicAssignment(fooTopicId, 0, 1, 2))) - .build()) - .withAssignment(memberId1, mkAssignment( - mkTopicAssignment(fooTopicId, 0, 1, 2))) - .withAssignmentEpoch(10)) - .build(); - - CoordinatorResult result; - - // Prepare new assignment for the group. - assignor.prepareGroupAssignment(new GroupAssignment( - new HashMap() { - { - put(memberId1, new MemberAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 0, 1) - ))); - put(memberId2, new MemberAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 2) - ))); - } - } - )); - - // Member 2 joins. - result = context.consumerGroupHeartbeat( - new ConsumerGroupHeartbeatRequestData() - .setGroupId(groupId) - .setMemberId(memberId2) - .setMemberEpoch(0) - .setRebalanceTimeoutMs(5000) - .setSubscribedTopicNames(Arrays.asList("foo", "bar")) - .setServerAssignor("range") - .setTopicPartitions(Collections.emptyList())); - - assertResponseEquals( - new ConsumerGroupHeartbeatResponseData() - .setMemberId(memberId2) - .setMemberEpoch(11) - .setHeartbeatIntervalMs(5000) - .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()), - result.response() - ); - - assertRecordEquals( - RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) - .setMemberEpoch(11) - .setPreviousMemberEpoch(0) - .setTargetMemberEpoch(11) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 2))) - .build()), - result.records().get(result.records().size() - 1) - ); - - assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId2)); - - // Member 1 heartbeats and transitions to Revoking. - result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() - .setGroupId(groupId) - .setMemberId(memberId1) - .setMemberEpoch(10)); - - assertResponseEquals( - new ConsumerGroupHeartbeatResponseData() - .setMemberId(memberId1) - .setMemberEpoch(10) - .setHeartbeatIntervalMs(5000) - .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(fooTopicId) - .setPartitions(Arrays.asList(0, 1))))), - result.response() - ); - - assertRecordsEquals(Collections.singletonList( - RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) - .setMemberEpoch(10) - .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(11) - .setAssignedPartitions(mkAssignment( - mkTopicAssignment(fooTopicId, 0, 1))) - .setPartitionsPendingRevocation(mkAssignment( - mkTopicAssignment(fooTopicId, 2))) - .build())), - result.records() - ); - - assertEquals(ConsumerGroupMember.MemberState.REVOKING, context.consumerGroupMemberState(groupId, memberId1)); - - // Prepare new assignment for the group. - assignor.prepareGroupAssignment(new GroupAssignment( - new HashMap() { - { - put(memberId1, new MemberAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 0) - ))); - put(memberId2, new MemberAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 2) - ))); - put(memberId3, new MemberAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 1) - ))); - } - } - )); - - // Member 3 joins. - result = context.consumerGroupHeartbeat( - new ConsumerGroupHeartbeatRequestData() - .setGroupId(groupId) - .setMemberId(memberId3) - .setMemberEpoch(0) - .setRebalanceTimeoutMs(5000) - .setSubscribedTopicNames(Arrays.asList("foo", "bar")) - .setServerAssignor("range") - .setTopicPartitions(Collections.emptyList())); - - assertResponseEquals( - new ConsumerGroupHeartbeatResponseData() - .setMemberId(memberId3) - .setMemberEpoch(12) - .setHeartbeatIntervalMs(5000) - .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()), - result.response() - ); - - assertRecordEquals( - RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) - .setMemberEpoch(12) - .setPreviousMemberEpoch(0) - .setTargetMemberEpoch(12) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 1))) - .build()), - result.records().get(result.records().size() - 1) - ); - - assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId3)); - - // When member 1 heartbeats, it transitions to Revoke again but an updated state. - result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() - .setGroupId(groupId) - .setMemberId(memberId1) - .setMemberEpoch(10)); - - assertResponseEquals( - new ConsumerGroupHeartbeatResponseData() - .setMemberId(memberId1) - .setMemberEpoch(10) - .setHeartbeatIntervalMs(5000) - .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(fooTopicId) - .setPartitions(Collections.singletonList(0))))), - result.response() - ); - - assertRecordsEquals(Collections.singletonList( - RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) - .setMemberEpoch(10) - .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(12) - .setAssignedPartitions(mkAssignment( - mkTopicAssignment(fooTopicId, 0))) - .setPartitionsPendingRevocation(mkAssignment( - mkTopicAssignment(fooTopicId, 1, 2))) - .build())), - result.records() - ); - - assertEquals(ConsumerGroupMember.MemberState.REVOKING, context.consumerGroupMemberState(groupId, memberId1)); - - // When member 2 heartbeats, it transitions to Assign again but with an updated state. - result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() - .setGroupId(groupId) - .setMemberId(memberId2) - .setMemberEpoch(11)); - - assertResponseEquals( - new ConsumerGroupHeartbeatResponseData() - .setMemberId(memberId2) - .setMemberEpoch(12) - .setHeartbeatIntervalMs(5000) - .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()), - result.response() - ); - - assertRecordsEquals(Collections.singletonList( - RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) - .setMemberEpoch(12) - .setPreviousMemberEpoch(11) - .setTargetMemberEpoch(12) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 2))) - .build())), - result.records() - ); - - assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId2)); - } - @Test public void testNewMemberIsRejectedWithMaximumMembersIsReached() { String groupId = "fooup"; @@ -2350,9 +2205,9 @@ public void testNewMemberIsRejectedWithMaximumMembersIsReached() { .withConsumerGroupMaxSize(2) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -2363,9 +2218,9 @@ public void testNewMemberIsRejectedWithMaximumMembersIsReached() { mkTopicAssignment(barTopicId, 0, 1))) .build()) .withMember(new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -2412,6 +2267,7 @@ public void testConsumerGroupStates() { assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY, context.consumerGroupState(groupId)); context.replay(RecordHelpers.newMemberSubscriptionRecord(groupId, new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setSubscribedTopicNames(Collections.singletonList(fooTopicName)) .build())); context.replay(RecordHelpers.newGroupEpochRecord(groupId, 11)); @@ -2425,19 +2281,18 @@ public void testConsumerGroupStates() { assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); context.replay(RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.UNREVOKED_PARTITIONS) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(11) - .setAssignedPartitions(mkAssignment(mkTopicAssignment(fooTopicId, 1, 2))) - .setPartitionsPendingAssignment(mkAssignment(mkTopicAssignment(fooTopicId, 3))) + .setAssignedPartitions(mkAssignment(mkTopicAssignment(fooTopicId, 1, 2, 3))) .build())); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); context.replay(RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment(mkTopicAssignment(fooTopicId, 1, 2, 3))) .build())); @@ -2502,9 +2357,9 @@ public void testSubscriptionMetadataRefreshedAfterGroupIsLoaded() { .build()) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId) + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -2527,7 +2382,7 @@ public void testSubscriptionMetadataRefreshedAfterGroupIsLoaded() { // The metadata refresh flag should be true. ConsumerGroup consumerGroup = context.groupMetadataManager - .getOrMaybeCreateConsumerGroup(groupId, false); + .consumerGroup(groupId); assertTrue(consumerGroup.hasMetadataExpired(context.time.milliseconds())); // Prepare the assignment result. @@ -2560,9 +2415,9 @@ public void testSubscriptionMetadataRefreshedAfterGroupIsLoaded() { ); ConsumerGroupMember expectedMember = new ConsumerGroupMember.Builder(memberId) + .setState(MemberState.STABLE) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(11) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -2613,9 +2468,9 @@ public void testSubscriptionMetadataRefreshedAgainAfterWriteFailure() { .build()) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId) + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -2638,7 +2493,7 @@ public void testSubscriptionMetadataRefreshedAgainAfterWriteFailure() { // The metadata refresh flag should be true. ConsumerGroup consumerGroup = context.groupMetadataManager - .getOrMaybeCreateConsumerGroup(groupId, false); + .consumerGroup(groupId); assertTrue(consumerGroup.hasMetadataExpired(context.time.milliseconds())); // Prepare the assignment result. @@ -2689,9 +2544,9 @@ public void testSubscriptionMetadataRefreshedAgainAfterWriteFailure() { ); ConsumerGroupMember expectedMember = new ConsumerGroupMember.Builder(memberId) + .setState(MemberState.STABLE) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(11) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -2876,7 +2731,7 @@ public void testOnNewMetadataImage() { // Ensures that all refresh flags are set to the future. Arrays.asList("group1", "group2", "group3", "group4", "group5").forEach(groupId -> { - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroup group = context.groupMetadataManager.consumerGroup(groupId); group.setMetadataRefreshDeadline(context.time.milliseconds() + 5000L, 0); assertFalse(group.hasMetadataExpired(context.time.milliseconds())); }); @@ -2913,12 +2768,12 @@ public void testOnNewMetadataImage() { // Verify the groups. Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> { - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroup group = context.groupMetadataManager.consumerGroup(groupId); assertTrue(group.hasMetadataExpired(context.time.milliseconds())); }); Collections.singletonList("group5").forEach(groupId -> { - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroup group = context.groupMetadataManager.consumerGroup(groupId); assertFalse(group.hasMetadataExpired(context.time.milliseconds())); }); @@ -2998,7 +2853,7 @@ public void testSessionTimeoutLifecycle() { // Verify that there are no timers. context.assertNoSessionTimeout(groupId, memberId); - context.assertNoRevocationTimeout(groupId, memberId); + context.assertNoRebalanceTimeout(groupId, memberId); } @Test @@ -3062,7 +2917,7 @@ public void testSessionTimeoutExpiration() { // Verify that there are no timers. context.assertNoSessionTimeout(groupId, memberId); - context.assertNoRevocationTimeout(groupId, memberId); + context.assertNoRebalanceTimeout(groupId, memberId); } @Test @@ -3143,16 +2998,15 @@ public void testSessionTimeoutExpirationStaticMember() { // Verify that there are no timers. context.assertNoSessionTimeout(groupId, memberId); - context.assertNoRevocationTimeout(groupId, memberId); + context.assertNoRebalanceTimeout(groupId, memberId); } @Test - public void testRevocationTimeoutLifecycle() { + public void testRebalanceTimeoutLifecycle() { String groupId = "fooup"; // Use a static member id as it makes the test easier. String memberId1 = Uuid.randomUuid().toString(); String memberId2 = Uuid.randomUuid().toString(); - String memberId3 = Uuid.randomUuid().toString(); Uuid fooTopicId = Uuid.randomUuid(); String fooTopicName = "foo"; @@ -3243,7 +3097,7 @@ public void testRevocationTimeoutLifecycle() { context.sleep(result.response().heartbeatIntervalMs()) ); - // Member 1 heartbeats and transitions to revoking. The revocation timeout + // Member 1 heartbeats and transitions to unrevoked partitions. The rebalance timeout // is scheduled. result = context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() @@ -3266,82 +3120,10 @@ public void testRevocationTimeoutLifecycle() { result.response() ); - // Verify that there is a revocation timeout. - context.assertRevocationTimeout(groupId, memberId1, 12000); - - assertEquals( - Collections.emptyList(), - context.sleep(result.response().heartbeatIntervalMs()) - ); - - // Prepare next assignment. - assignor.prepareGroupAssignment(new GroupAssignment( - new HashMap() { - { - put(memberId1, new MemberAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 0) - ))); - put(memberId2, new MemberAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 2) - ))); - put(memberId3, new MemberAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 1) - ))); - } - } - )); - - // Member 3 joins the group. - result = context.consumerGroupHeartbeat( - new ConsumerGroupHeartbeatRequestData() - .setGroupId(groupId) - .setMemberId(memberId3) - .setMemberEpoch(0) - .setRebalanceTimeoutMs(90000) - .setSubscribedTopicNames(Collections.singletonList("foo")) - .setTopicPartitions(Collections.emptyList())); - - assertResponseEquals( - new ConsumerGroupHeartbeatResponseData() - .setMemberId(memberId3) - .setMemberEpoch(3) - .setHeartbeatIntervalMs(5000) - .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()), - result.response() - ); - - assertEquals( - Collections.emptyList(), - context.sleep(result.response().heartbeatIntervalMs()) - ); - - // Member 1 heartbeats and re-transitions to revoking. The revocation timeout - // is re-scheduled. - result = context.consumerGroupHeartbeat( - new ConsumerGroupHeartbeatRequestData() - .setGroupId(groupId) - .setMemberId(memberId1) - .setMemberEpoch(1) - .setRebalanceTimeoutMs(90000) - .setSubscribedTopicNames(Collections.singletonList("foo"))); - - assertResponseEquals( - new ConsumerGroupHeartbeatResponseData() - .setMemberId(memberId1) - .setMemberEpoch(1) - .setHeartbeatIntervalMs(5000) - .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(fooTopicId) - .setPartitions(Collections.singletonList(0))))), - result.response() - ); - // Verify that there is a revocation timeout. Keep a reference // to the timeout for later. ScheduledTimeout scheduledTimeout = - context.assertRevocationTimeout(groupId, memberId1, 90000); + context.assertRebalanceTimeout(groupId, memberId1, 12000); assertEquals( Collections.emptyList(), @@ -3356,23 +3138,18 @@ public void testRevocationTimeoutLifecycle() { .setMemberEpoch(1) .setTopicPartitions(Collections.singletonList(new ConsumerGroupHeartbeatRequestData.TopicPartitions() .setTopicId(fooTopicId) - .setPartitions(Collections.singletonList(0))))); + .setPartitions(Arrays.asList(0, 1))))); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() .setMemberId(memberId1) - .setMemberEpoch(3) - .setHeartbeatIntervalMs(5000) - .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(fooTopicId) - .setPartitions(Collections.singletonList(0))))), + .setMemberEpoch(2) + .setHeartbeatIntervalMs(5000), result.response() ); // Verify that there is not revocation timeout. - context.assertNoRevocationTimeout(groupId, memberId1); + context.assertNoRebalanceTimeout(groupId, memberId1); // Execute the scheduled revocation timeout captured earlier to simulate a // stale timeout. This should be a no-op. @@ -3380,7 +3157,7 @@ public void testRevocationTimeoutLifecycle() { } @Test - public void testRevocationTimeoutExpiration() { + public void testRebalanceTimeoutExpiration() { String groupId = "fooup"; // Use a static member id as it makes the test easier. String memberId1 = Uuid.randomUuid().toString(); @@ -3502,7 +3279,7 @@ public void testRevocationTimeoutExpiration() { // Verify the expired timeout. assertEquals( Collections.singletonList(new ExpiredTimeout( - consumerGroupRevocationTimeoutKey(groupId, memberId1), + consumerGroupRebalanceTimeoutKey(groupId, memberId1), new CoordinatorResult<>( Arrays.asList( RecordHelpers.newCurrentAssignmentTombstoneRecord(groupId, memberId1), @@ -3517,7 +3294,7 @@ public void testRevocationTimeoutExpiration() { // Verify that there are no timers. context.assertNoSessionTimeout(groupId, memberId1); - context.assertNoRevocationTimeout(groupId, memberId1); + context.assertNoRebalanceTimeout(groupId, memberId1); } @Test @@ -3535,9 +3312,9 @@ public void testOnLoaded() { .build()) .withConsumerGroup(new ConsumerGroupBuilder("foo", 10) .withMember(new ConsumerGroupMember.Builder("foo-1") + .setState(MemberState.UNREVOKED_PARTITIONS) .setMemberEpoch(9) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Collections.singletonList("foo")) @@ -3548,19 +3325,15 @@ public void testOnLoaded() { mkTopicAssignment(fooTopicId, 3, 4, 5))) .build()) .withMember(new ConsumerGroupMember.Builder("foo-2") + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Collections.singletonList("foo")) .setServerAssignorName("range") - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 3, 4, 5))) .build()) .withAssignment("foo-1", mkAssignment( - mkTopicAssignment(fooTopicId, 0, 1, 2))) - .withAssignment("foo-2", mkAssignment( mkTopicAssignment(fooTopicId, 3, 4, 5))) .withAssignmentEpoch(10)) .build(); @@ -3574,7 +3347,7 @@ public void testOnLoaded() { assertNotNull(context.timer.timeout(consumerGroupSessionTimeoutKey("foo", "foo-2"))); // foo-1 should also have a revocation timeout in place. - assertNotNull(context.timer.timeout(consumerGroupRevocationTimeoutKey("foo", "foo-1"))); + assertNotNull(context.timer.timeout(consumerGroupRebalanceTimeoutKey("foo", "foo-1"))); } @Test @@ -9394,7 +9167,7 @@ public void testClassicGroupDelete() { List expectedRecords = Collections.singletonList(RecordHelpers.newGroupMetadataTombstoneRecord("group-id")); List records = new ArrayList<>(); - context.groupMetadataManager.deleteGroup("group-id", records); + context.groupMetadataManager.createGroupTombstoneRecords("group-id", records); assertEquals(expectedRecords, records); } @@ -9421,43 +9194,43 @@ public void testClassicGroupMaybeDelete() { @Test public void testConsumerGroupDelete() { + String groupId = "group-id"; GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)) .build(); - context.groupMetadataManager.getOrMaybeCreateConsumerGroup("group-id", true); List expectedRecords = Arrays.asList( - RecordHelpers.newTargetAssignmentEpochTombstoneRecord("group-id"), - RecordHelpers.newGroupSubscriptionMetadataTombstoneRecord("group-id"), - RecordHelpers.newGroupEpochTombstoneRecord("group-id") + RecordHelpers.newTargetAssignmentEpochTombstoneRecord(groupId), + RecordHelpers.newGroupSubscriptionMetadataTombstoneRecord(groupId), + RecordHelpers.newGroupEpochTombstoneRecord(groupId) ); List records = new ArrayList<>(); - context.groupMetadataManager.deleteGroup("group-id", records); + context.groupMetadataManager.createGroupTombstoneRecords("group-id", records); assertEquals(expectedRecords, records); } @Test public void testConsumerGroupMaybeDelete() { + String groupId = "group-id"; GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)) .build(); - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup("group-id", true); List expectedRecords = Arrays.asList( - RecordHelpers.newTargetAssignmentEpochTombstoneRecord("group-id"), - RecordHelpers.newGroupSubscriptionMetadataTombstoneRecord("group-id"), - RecordHelpers.newGroupEpochTombstoneRecord("group-id") + RecordHelpers.newTargetAssignmentEpochTombstoneRecord(groupId), + RecordHelpers.newGroupSubscriptionMetadataTombstoneRecord(groupId), + RecordHelpers.newGroupEpochTombstoneRecord(groupId) ); List records = new ArrayList<>(); - context.groupMetadataManager.maybeDeleteGroup("group-id", records); + context.groupMetadataManager.maybeDeleteGroup(groupId, records); assertEquals(expectedRecords, records); records = new ArrayList<>(); - group.updateMember(new ConsumerGroupMember.Builder("member") + context.replay(RecordHelpers.newMemberSubscriptionRecord(groupId, new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) - .setTargetMemberEpoch(10) .setPreviousMemberEpoch(10) - .build() - ); - context.groupMetadataManager.maybeDeleteGroup("group-id", records); + .build())); + context.groupMetadataManager.maybeDeleteGroup(groupId, records); assertEquals(Collections.emptyList(), records); } @@ -9582,7 +9355,7 @@ public void testOnConsumerGroupStateTransition() { verify(context.metrics, times(1)).onConsumerGroupStateTransition(ConsumerGroup.ConsumerGroupState.EMPTY, null); // Replaying a tombstone for a group that has already been removed should not decrement metric. - tombstones.forEach(tombstone -> assertThrows(GroupIdNotFoundException.class, () -> context.replay(tombstone))); + tombstones.forEach(tombstone -> assertThrows(IllegalStateException.class, () -> context.replay(tombstone))); verify(context.metrics, times(1)).onConsumerGroupStateTransition(ConsumerGroup.ConsumerGroupState.EMPTY, null); } @@ -9599,14 +9372,154 @@ public void testOnConsumerGroupStateTransitionOnLoading() { context.replay(RecordHelpers.newTargetAssignmentEpochTombstoneRecord("group-id")); context.replay(RecordHelpers.newGroupEpochTombstoneRecord("group-id")); IntStream.range(0, 3).forEach(__ -> { - assertThrows(GroupIdNotFoundException.class, () -> context.replay(RecordHelpers.newTargetAssignmentEpochTombstoneRecord("group-id"))); - assertThrows(GroupIdNotFoundException.class, () -> context.replay(RecordHelpers.newGroupEpochTombstoneRecord("group-id"))); + assertThrows(IllegalStateException.class, () -> context.replay(RecordHelpers.newTargetAssignmentEpochTombstoneRecord("group-id"))); + assertThrows(IllegalStateException.class, () -> context.replay(RecordHelpers.newGroupEpochTombstoneRecord("group-id"))); }); verify(context.metrics, times(1)).onConsumerGroupStateTransition(null, ConsumerGroup.ConsumerGroupState.EMPTY); verify(context.metrics, times(1)).onConsumerGroupStateTransition(ConsumerGroup.ConsumerGroupState.EMPTY, null); } + @Test + public void testConsumerGroupHeartbeatWithNonEmptyClassicGroup() { + String classicGroupId = "classic-group-id"; + String memberId = Uuid.randomUuid().toString(); + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + assignor.prepareGroupAssignment(new GroupAssignment(Collections.emptyMap())); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .build(); + ClassicGroup classicGroup = new ClassicGroup( + new LogContext(), + classicGroupId, + EMPTY, + context.time, + context.metrics + ); + context.replay(RecordHelpers.newGroupMetadataRecord(classicGroup, classicGroup.groupAssignment(), MetadataVersion.latestTesting())); + + context.groupMetadataManager.getOrMaybeCreateClassicGroup(classicGroupId, false).transitionTo(PREPARING_REBALANCE); + assertThrows(GroupIdNotFoundException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(classicGroupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList()))); + } + + @Test + public void testConsumerGroupHeartbeatWithEmptyClassicGroup() { + String classicGroupId = "classic-group-id"; + String memberId = Uuid.randomUuid().toString(); + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + assignor.prepareGroupAssignment(new GroupAssignment(Collections.emptyMap())); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .build(); + ClassicGroup classicGroup = new ClassicGroup( + new LogContext(), + classicGroupId, + EMPTY, + context.time, + context.metrics + ); + context.replay(RecordHelpers.newGroupMetadataRecord(classicGroup, classicGroup.groupAssignment(), MetadataVersion.latestTesting())); + + CoordinatorResult result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(classicGroupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList())); + + ConsumerGroupMember expectedMember = new ConsumerGroupMember.Builder(memberId) + .setState(MemberState.STABLE) + .setMemberEpoch(1) + .setPreviousMemberEpoch(0) + .setRebalanceTimeoutMs(5000) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(Collections.emptyMap()) + .build(); + + assertEquals(Errors.NONE.code(), result.response().errorCode()); + assertEquals( + Arrays.asList( + RecordHelpers.newGroupMetadataTombstoneRecord(classicGroupId), + RecordHelpers.newMemberSubscriptionRecord(classicGroupId, expectedMember), + RecordHelpers.newGroupEpochRecord(classicGroupId, 1), + RecordHelpers.newTargetAssignmentRecord(classicGroupId, memberId, Collections.emptyMap()), + RecordHelpers.newTargetAssignmentEpochRecord(classicGroupId, 1), + RecordHelpers.newCurrentAssignmentRecord(classicGroupId, expectedMember) + ), + result.records() + ); + assertEquals( + Group.GroupType.CONSUMER, + context.groupMetadataManager.consumerGroup(classicGroupId).type() + ); + } + + @Test + public void testClassicGroupJoinWithNonEmptyConsumerGroup() throws Exception { + String consumerGroupId = "consumer-group-id"; + String memberId = Uuid.randomUuid().toString(); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroup(new ConsumerGroupBuilder(consumerGroupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setState(MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(10) + .build())) + .build(); + + JoinGroupRequestData request = new GroupMetadataManagerTestContext.JoinGroupRequestBuilder() + .withGroupId(consumerGroupId) + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .build(); + + GroupMetadataManagerTestContext.JoinResult joinResult = context.sendClassicGroupJoin(request); + assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), joinResult.joinFuture.get().errorCode()); + } + + @Test + public void testClassicGroupJoinWithEmptyConsumerGroup() throws Exception { + String consumerGroupId = "consumer-group-id"; + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroup(new ConsumerGroupBuilder(consumerGroupId, 10)) + .build(); + + JoinGroupRequestData request = new GroupMetadataManagerTestContext.JoinGroupRequestBuilder() + .withGroupId(consumerGroupId) + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .build(); + GroupMetadataManagerTestContext.JoinResult joinResult = context.sendClassicGroupJoin(request, true); + + List expectedRecords = Arrays.asList( + RecordHelpers.newTargetAssignmentEpochTombstoneRecord(consumerGroupId), + RecordHelpers.newGroupSubscriptionMetadataTombstoneRecord(consumerGroupId), + RecordHelpers.newGroupEpochTombstoneRecord(consumerGroupId) + ); + + assertEquals(Errors.MEMBER_ID_REQUIRED.code(), joinResult.joinFuture.get().errorCode()); + assertEquals(expectedRecords, joinResult.records.subList(0, expectedRecords.size())); + assertEquals( + Group.GroupType.CLASSIC, + context.groupMetadataManager.getOrMaybeCreateClassicGroup(consumerGroupId, false).type() + ); + } + private static void checkJoinGroupResponse( JoinGroupResponseData expectedResponse, JoinGroupResponseData actualResponse, diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java index d2d00c1582..4a7cd8cae9 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java @@ -47,7 +47,7 @@ import org.apache.kafka.coordinator.group.classic.ClassicGroup; import org.apache.kafka.coordinator.group.consumer.ConsumerGroup; import org.apache.kafka.coordinator.group.consumer.ConsumerGroupBuilder; -import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember; +import org.apache.kafka.coordinator.group.consumer.MemberState; import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey; import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue; import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataKey; @@ -85,7 +85,7 @@ import static org.apache.kafka.common.requests.JoinGroupRequest.UNKNOWN_MEMBER_ID; import static org.apache.kafka.coordinator.group.GroupMetadataManager.EMPTY_RESULT; import static org.apache.kafka.coordinator.group.GroupMetadataManager.classicGroupHeartbeatKey; -import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupRevocationTimeoutKey; +import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupRebalanceTimeoutKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupSessionTimeoutKey; import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.COMPLETING_REBALANCE; import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.DEAD; @@ -483,16 +483,16 @@ public ConsumerGroup.ConsumerGroupState consumerGroupState( String groupId ) { return groupMetadataManager - .getOrMaybeCreateConsumerGroup(groupId, false) + .consumerGroup(groupId) .state(); } - public ConsumerGroupMember.MemberState consumerGroupMemberState( + public MemberState consumerGroupMemberState( String groupId, String memberId ) { return groupMetadataManager - .getOrMaybeCreateConsumerGroup(groupId, false) + .consumerGroup(groupId) .getOrMaybeCreateMember(memberId, false) .state(); } @@ -556,24 +556,24 @@ public void assertNoSessionTimeout( assertNull(timeout); } - public MockCoordinatorTimer.ScheduledTimeout assertRevocationTimeout( + public MockCoordinatorTimer.ScheduledTimeout assertRebalanceTimeout( String groupId, String memberId, long delayMs ) { MockCoordinatorTimer.ScheduledTimeout timeout = - timer.timeout(consumerGroupRevocationTimeoutKey(groupId, memberId)); + timer.timeout(consumerGroupRebalanceTimeoutKey(groupId, memberId)); assertNotNull(timeout); assertEquals(time.milliseconds() + delayMs, timeout.deadlineMs); return timeout; } - public void assertNoRevocationTimeout( + public void assertNoRebalanceTimeout( String groupId, String memberId ) { MockCoordinatorTimer.ScheduledTimeout timeout = - timer.timeout(consumerGroupRevocationTimeoutKey(groupId, memberId)); + timer.timeout(consumerGroupRebalanceTimeoutKey(groupId, memberId)); assertNull(timeout); } @@ -643,7 +643,7 @@ public JoinGroupResponseData joinClassicGroupAsDynamicMemberAndCompleteRebalance JoinGroupResponseData leaderJoinResponse = joinClassicGroupAsDynamicMemberAndCompleteJoin(new JoinGroupRequestBuilder() - .withGroupId("group-id") + .withGroupId(groupId) .withMemberId(UNKNOWN_MEMBER_ID) .withDefaultProtocolTypeAndProtocols() .withRebalanceTimeoutMs(10000) @@ -654,7 +654,7 @@ public JoinGroupResponseData joinClassicGroupAsDynamicMemberAndCompleteRebalance assertTrue(group.isInState(COMPLETING_REBALANCE)); SyncResult syncResult = sendClassicGroupSync(new SyncGroupRequestBuilder() - .withGroupId("group-id") + .withGroupId(groupId) .withMemberId(leaderJoinResponse.memberId()) .withGenerationId(leaderJoinResponse.generationId()) .build()); @@ -804,7 +804,7 @@ public RebalanceResult staticMembersJoinAndRebalance( int rebalanceTimeoutMs, int sessionTimeoutMs ) throws Exception { - ClassicGroup group = createClassicGroup("group-id"); + ClassicGroup group = createClassicGroup(groupId); JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() .withGroupId(groupId) @@ -901,7 +901,7 @@ public RebalanceResult staticMembersJoinAndRebalance( public PendingMemberGroupResult setupGroupWithPendingMember(ClassicGroup group) throws Exception { // Add the first member JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() - .withGroupId("group-id") + .withGroupId(group.groupId()) .withMemberId(UNKNOWN_MEMBER_ID) .withDefaultProtocolTypeAndProtocols() .withRebalanceTimeoutMs(10000) @@ -914,7 +914,7 @@ public PendingMemberGroupResult setupGroupWithPendingMember(ClassicGroup group) List assignment = new ArrayList<>(); assignment.add(new SyncGroupRequestData.SyncGroupRequestAssignment().setMemberId(leaderJoinResponse.memberId())); SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() - .withGroupId("group-id") + .withGroupId(group.groupId()) .withMemberId(leaderJoinResponse.memberId()) .withGenerationId(leaderJoinResponse.generationId()) .withAssignment(assignment) @@ -1191,7 +1191,7 @@ public void verifyDescribeGroupsReturnsDeadGroup(String groupId) { assertEquals( Collections.singletonList(new DescribeGroupsResponseData.DescribedGroup() - .setGroupId("group-id") + .setGroupId(groupId) .setGroupState(DEAD.toString()) ), describedGroups diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java index 87f7b47103..4b3d76bbd1 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java @@ -194,7 +194,7 @@ public Group getOrMaybeCreateGroup( true ); case CONSUMER: - return groupMetadataManager.getOrMaybeCreateConsumerGroup( + return groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( groupId, true ); @@ -1079,7 +1079,7 @@ public void testConsumerGroupOffsetCommitWithUnknownMemberId() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); // Create an empty group. - context.groupMetadataManager.getOrMaybeCreateConsumerGroup( + context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( "foo", true ); @@ -1108,7 +1108,7 @@ public void testConsumerGroupOffsetCommitWithStaleMemberEpoch() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); // Create an empty group. - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup( + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( "foo", true ); @@ -1116,7 +1116,6 @@ public void testConsumerGroupOffsetCommitWithStaleMemberEpoch() { // Add member. group.updateMember(new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) - .setTargetMemberEpoch(10) .setPreviousMemberEpoch(10) .build() ); @@ -1154,7 +1153,7 @@ public void testConsumerGroupOffsetCommitWithVersionSmallerThanVersion9(short ve OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); // Create an empty group. - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup( + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( "foo", true ); @@ -1162,7 +1161,6 @@ public void testConsumerGroupOffsetCommitWithVersionSmallerThanVersion9(short ve // Add member. group.updateMember(new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) - .setTargetMemberEpoch(10) .setPreviousMemberEpoch(10) .build() ); @@ -1192,7 +1190,7 @@ public void testConsumerGroupOffsetCommitFromAdminClient() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); // Create an empty group. - context.groupMetadataManager.getOrMaybeCreateConsumerGroup( + context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( "foo", true ); @@ -1248,7 +1246,7 @@ public void testConsumerGroupOffsetCommit() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); // Create an empty group. - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup( + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( "foo", true ); @@ -1256,7 +1254,6 @@ public void testConsumerGroupOffsetCommit() { // Add member. group.updateMember(new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) - .setTargetMemberEpoch(10) .setPreviousMemberEpoch(10) .build() ); @@ -1319,7 +1316,7 @@ public void testConsumerGroupOffsetCommitWithOffsetMetadataTooLarge() { .build(); // Create an empty group. - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup( + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( "foo", true ); @@ -1327,7 +1324,6 @@ public void testConsumerGroupOffsetCommitWithOffsetMetadataTooLarge() { // Add member. group.updateMember(new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) - .setTargetMemberEpoch(10) .setPreviousMemberEpoch(10) .build() ); @@ -1397,7 +1393,7 @@ public void testConsumerGroupTransactionalOffsetCommit() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); // Create an empty group. - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup( + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( "foo", true ); @@ -1405,7 +1401,6 @@ public void testConsumerGroupTransactionalOffsetCommit() { // Add member. group.updateMember(new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) - .setTargetMemberEpoch(10) .setPreviousMemberEpoch(10) .build() ); @@ -1488,7 +1483,7 @@ public void testConsumerGroupTransactionalOffsetCommitWithUnknownMemberId() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); // Create an empty group. - context.groupMetadataManager.getOrMaybeCreateConsumerGroup( + context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( "foo", true ); @@ -1517,7 +1512,7 @@ public void testConsumerGroupTransactionalOffsetCommitWithStaleMemberEpoch() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); // Create an empty group. - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup( + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( "foo", true ); @@ -1525,7 +1520,6 @@ public void testConsumerGroupTransactionalOffsetCommitWithStaleMemberEpoch() { // Add member. group.updateMember(new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) - .setTargetMemberEpoch(10) .setPreviousMemberEpoch(10) .build() ); @@ -1781,7 +1775,7 @@ public void testFetchOffsetsWithUnknownGroup() { public void testFetchOffsetsAtDifferentCommittedOffset() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); - context.groupMetadataManager.getOrMaybeCreateConsumerGroup("group", true); + context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup("group", true); assertEquals(0, context.lastWrittenOffset); context.commitOffset("group", "foo", 0, 100L, 1); @@ -1922,7 +1916,7 @@ public void testFetchOffsetsAtDifferentCommittedOffset() { public void testFetchOffsetsWithPendingTransactionalOffsets() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); - context.groupMetadataManager.getOrMaybeCreateConsumerGroup("group", true); + context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup("group", true); context.commitOffset("group", "foo", 0, 100L, 1); context.commitOffset("group", "foo", 1, 110L, 1); @@ -2027,7 +2021,7 @@ public void testFetchAllOffsetsWithUnknownGroup() { public void testFetchAllOffsetsAtDifferentCommittedOffset() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); - context.groupMetadataManager.getOrMaybeCreateConsumerGroup("group", true); + context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup("group", true); assertEquals(0, context.lastWrittenOffset); context.commitOffset("group", "foo", 0, 100L, 1); @@ -2114,7 +2108,7 @@ public void testFetchAllOffsetsAtDifferentCommittedOffset() { public void testFetchAllOffsetsWithPendingTransactionalOffsets() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); - context.groupMetadataManager.getOrMaybeCreateConsumerGroup("group", true); + context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup("group", true); context.commitOffset("group", "foo", 0, 100L, 1); context.commitOffset("group", "foo", 1, 110L, 1); @@ -2188,7 +2182,7 @@ public void testFetchAllOffsetsWithPendingTransactionalOffsets() { public void testConsumerGroupOffsetFetchWithMemberIdAndEpoch() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); // Create consumer group. - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup("group", true); + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup("group", true); // Create member. group.getOrMaybeCreateMember("member", true); // Commit offset. @@ -2223,7 +2217,7 @@ public void testConsumerGroupOffsetFetchWithMemberIdAndEpoch() { public void testConsumerGroupOffsetFetchFromAdminClient() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); // Create consumer group. - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup("group", true); + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup("group", true); // Create member. group.getOrMaybeCreateMember("member", true); // Commit offset. @@ -2257,7 +2251,7 @@ public void testConsumerGroupOffsetFetchFromAdminClient() { @Test public void testConsumerGroupOffsetFetchWithUnknownMemberId() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); - context.groupMetadataManager.getOrMaybeCreateConsumerGroup("group", true); + context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup("group", true); // Fetch offsets case. List topics = Collections.singletonList( @@ -2282,7 +2276,7 @@ public void testConsumerGroupOffsetFetchWithUnknownMemberId() { @Test public void testConsumerGroupOffsetFetchWithStaleMemberEpoch() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup("group", true); + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup("group", true); group.getOrMaybeCreateMember("member", true); // Fetch offsets case. @@ -2346,7 +2340,7 @@ public void testGenericGroupOffsetDeleteWithPendingTransactionalOffsets() { @Test public void testConsumerGroupOffsetDelete() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup( + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( "foo", true ); @@ -2358,7 +2352,7 @@ public void testConsumerGroupOffsetDelete() { @Test public void testConsumerGroupOffsetDeleteWithErrors() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup( + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( "foo", true ); @@ -2388,7 +2382,7 @@ public void testConsumerGroupOffsetDeleteWithErrors() { @Test public void testConsumerGroupOffsetDeleteWithPendingTransactionalOffsets() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); - ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup( + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( "foo", true ); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java index 2a15fb7dec..17d24959a1 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java @@ -22,10 +22,9 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.coordinator.group.consumer.ClientAssignor; import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember; +import org.apache.kafka.coordinator.group.consumer.MemberState; import org.apache.kafka.coordinator.group.consumer.TopicMetadata; -import org.apache.kafka.coordinator.group.consumer.VersionedMetadata; import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey; import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue; import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataKey; @@ -54,8 +53,6 @@ import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -104,14 +101,6 @@ public void testNewMemberSubscriptionRecord() { .setSubscribedTopicNames(Arrays.asList("foo", "zar", "bar")) .setSubscribedTopicRegex("regex") .setServerAssignorName("range") - .setClientAssignors(Collections.singletonList(new ClientAssignor( - "assignor", - (byte) 0, - (byte) 1, - (byte) 10, - new VersionedMetadata( - (byte) 5, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)))))) .build(); Record expectedRecord = new Record( @@ -129,13 +118,7 @@ public void testNewMemberSubscriptionRecord() { .setClientHost("client-host") .setSubscribedTopicNames(Arrays.asList("bar", "foo", "zar")) .setSubscribedTopicRegex("regex") - .setServerAssignor("range") - .setAssignors(Collections.singletonList(new ConsumerGroupMemberMetadataValue.Assignor() - .setName("assignor") - .setMinimumVersion((short) 1) - .setMaximumVersion((short) 10) - .setVersion((short) 5) - .setMetadata("hello".getBytes(StandardCharsets.UTF_8)))), + .setServerAssignor("range"), (short) 0)); assertEquals(expectedRecord, newMemberSubscriptionRecord( @@ -397,11 +380,6 @@ public void testNewCurrentAssignmentRecord() { mkSortedTopicAssignment(topicId2, 24, 25, 26) ); - Map> assigning = mkSortedAssignment( - mkSortedTopicAssignment(topicId1, 17, 18, 19), - mkSortedTopicAssignment(topicId2, 27, 28, 29) - ); - Record expectedRecord = new Record( new ApiMessageAndVersion( new ConsumerGroupCurrentMemberAssignmentKey() @@ -410,9 +388,9 @@ public void testNewCurrentAssignmentRecord() { (short) 8), new ApiMessageAndVersion( new ConsumerGroupCurrentMemberAssignmentValue() + .setState(MemberState.UNREVOKED_PARTITIONS.value()) .setMemberEpoch(22) .setPreviousMemberEpoch(21) - .setTargetMemberEpoch(23) .setAssignedPartitions(Arrays.asList( new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() .setTopicId(topicId1) @@ -426,25 +404,17 @@ public void testNewCurrentAssignmentRecord() { .setPartitions(Arrays.asList(14, 15, 16)), new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() .setTopicId(topicId2) - .setPartitions(Arrays.asList(24, 25, 26)))) - .setPartitionsPendingAssignment(Arrays.asList( - new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() - .setTopicId(topicId1) - .setPartitions(Arrays.asList(17, 18, 19)), - new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() - .setTopicId(topicId2) - .setPartitions(Arrays.asList(27, 28, 29)))), + .setPartitions(Arrays.asList(24, 25, 26)))), (short) 0)); assertEquals(expectedRecord, newCurrentAssignmentRecord( "group-id", new ConsumerGroupMember.Builder("member-id") + .setState(MemberState.UNREVOKED_PARTITIONS) .setMemberEpoch(22) .setPreviousMemberEpoch(21) - .setTargetMemberEpoch(23) .setAssignedPartitions(assigned) .setPartitionsPendingRevocation(revoking) - .setPartitionsPendingAssignment(assigning) .build() )); } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/AssignmentTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/AssignmentTest.java index 2cff004ded..536ec71cac 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/AssignmentTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/AssignmentTest.java @@ -20,11 +20,8 @@ import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue; import org.junit.jupiter.api.Test; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -32,29 +29,13 @@ import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertThrows; public class AssignmentTest { @Test - public void testPartitionsAndMetadataCannotBeNull() { - assertThrows(NullPointerException.class, () -> new Assignment( - (byte) 1, - null, - new VersionedMetadata( - (short) 1, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)) - ) - )); - - assertThrows(NullPointerException.class, () -> new Assignment( - (byte) 1, - mkAssignment( - mkTopicAssignment(Uuid.randomUuid(), 1, 2, 3) - ), - null - )); + public void testPartitionsCannotBeNull() { + assertThrows(NullPointerException.class, () -> new Assignment(null)); } @Test @@ -62,21 +43,8 @@ public void testAttributes() { Map> partitions = mkAssignment( mkTopicAssignment(Uuid.randomUuid(), 1, 2, 3) ); - - VersionedMetadata metadata = new VersionedMetadata( - (short) 1, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)) - ); - - Assignment assignment = new Assignment( - (byte) 1, - partitions, - metadata - ); - - assertEquals((byte) 1, assignment.error()); + Assignment assignment = new Assignment(partitions); assertEquals(partitions, assignment.partitions()); - assertEquals(metadata, assignment.metadata()); } @Test @@ -93,22 +61,14 @@ public void testFromTargetAssignmentRecord() { .setPartitions(Arrays.asList(4, 5, 6))); ConsumerGroupTargetAssignmentMemberValue record = new ConsumerGroupTargetAssignmentMemberValue() - .setError((byte) 1) - .setTopicPartitions(partitions) - .setMetadataVersion((short) 2) - .setMetadataBytes("foo".getBytes(StandardCharsets.UTF_8)); + .setTopicPartitions(partitions); Assignment assignment = Assignment.fromRecord(record); - assertEquals((short) 1, assignment.error()); assertEquals(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3), mkTopicAssignment(topicId2, 4, 5, 6) ), assignment.partitions()); - assertEquals(new VersionedMetadata( - (short) 2, - ByteBuffer.wrap("foo".getBytes(StandardCharsets.UTF_8)) - ), assignment.metadata()); } @Test @@ -117,27 +77,6 @@ public void testEquals() { mkTopicAssignment(Uuid.randomUuid(), 1, 2, 3) ); - VersionedMetadata metadata = new VersionedMetadata( - (short) 1, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)) - ); - - Assignment assignment = new Assignment( - (byte) 1, - partitions, - metadata - ); - - assertEquals(new Assignment( - (byte) 1, - partitions, - metadata - ), assignment); - - assertNotEquals(new Assignment( - (byte) 1, - Collections.emptyMap(), - metadata - ), assignment); + assertEquals(new Assignment(partitions), new Assignment(partitions)); } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ClientAssignorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ClientAssignorTest.java deleted file mode 100644 index d136a411cd..0000000000 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ClientAssignorTest.java +++ /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 org.apache.kafka.coordinator.group.consumer; - -import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; -import org.junit.jupiter.api.Test; - -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; - -public class ClientAssignorTest { - - @Test - public void testNameAndMetadataCannotBeNull() { - assertThrows(NullPointerException.class, () -> new ClientAssignor( - "range", - (byte) 2, - (short) 5, - (short) 10, - null - )); - - assertThrows(NullPointerException.class, () -> new ClientAssignor( - null, - (byte) 2, - (short) 5, - (short) 10, - new VersionedMetadata( - (short) 8, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)) - ) - )); - } - - @Test - public void testAttributes() { - ClientAssignor clientAssignor = new ClientAssignor( - "range", - (byte) 2, - (short) 5, - (short) 10, - new VersionedMetadata( - (short) 8, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)) - ) - ); - - assertEquals("range", clientAssignor.name()); - assertEquals((byte) 2, clientAssignor.reason()); - assertEquals((short) 5, clientAssignor.minimumVersion()); - assertEquals((short) 10, clientAssignor.maximumVersion()); - assertEquals(new VersionedMetadata( - (short) 8, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)) - ), clientAssignor.metadata()); - } - - @Test - public void testFromRecord() { - ConsumerGroupMemberMetadataValue.Assignor record = new ConsumerGroupMemberMetadataValue.Assignor() - .setName("range") - .setReason((byte) 2) - .setMinimumVersion((byte) 5) - .setMaximumVersion((byte) 10) - .setVersion((byte) 8) - .setMetadata("hello".getBytes(StandardCharsets.UTF_8)); - - ClientAssignor clientAssignor = ClientAssignor.fromRecord(record); - - assertEquals("range", clientAssignor.name()); - assertEquals((byte) 2, clientAssignor.reason()); - assertEquals((short) 5, clientAssignor.minimumVersion()); - assertEquals((short) 10, clientAssignor.maximumVersion()); - assertEquals(new VersionedMetadata( - (short) 8, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)) - ), clientAssignor.metadata()); - } - - @Test - public void testEquals() { - ClientAssignor clientAssignor = new ClientAssignor( - "range", - (byte) 2, - (short) 5, - (short) 10, - new VersionedMetadata( - (short) 8, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)) - ) - ); - - assertEquals(new ClientAssignor( - "range", - (byte) 2, - (short) 5, - (short) 10, - new VersionedMetadata( - (short) 8, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)) - ) - ), clientAssignor); - - assertNotEquals(new ClientAssignor( - "uniform", - (byte) 2, - (short) 5, - (short) 10, - new VersionedMetadata( - (short) 8, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)) - ) - ), clientAssignor); - } -} diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMemberTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMemberTest.java index a5daf3f92b..cfb9a79a2d 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMemberTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMemberTest.java @@ -24,7 +24,6 @@ import org.apache.kafka.image.MetadataImage; import org.junit.jupiter.api.Test; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -40,6 +39,7 @@ import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; import static org.junit.jupiter.api.Assertions.assertEquals; + public class ConsumerGroupMemberTest { @Test @@ -51,7 +51,6 @@ public void testNewMember() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member-id") .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(11) .setInstanceId("instance-id") .setRackId("rack-id") .setRebalanceTimeoutMs(5000) @@ -60,27 +59,15 @@ public void testNewMember() { .setSubscribedTopicNames(Arrays.asList("foo", "bar")) .setSubscribedTopicRegex("regex") .setServerAssignorName("range") - .setClientAssignors(Collections.singletonList( - new ClientAssignor( - "assignor", - (byte) 0, - (byte) 0, - (byte) 1, - new VersionedMetadata( - (byte) 1, - ByteBuffer.allocate(0))))) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3))) .setPartitionsPendingRevocation(mkAssignment( mkTopicAssignment(topicId2, 4, 5, 6))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(topicId3, 7, 8, 9))) .build(); assertEquals("member-id", member.memberId()); assertEquals(10, member.memberEpoch()); assertEquals(9, member.previousMemberEpoch()); - assertEquals(11, member.targetMemberEpoch()); assertEquals("instance-id", member.instanceId()); assertEquals("rack-id", member.rackId()); assertEquals("client-id", member.clientId()); @@ -89,20 +76,8 @@ public void testNewMember() { assertEquals(Arrays.asList("bar", "foo"), member.subscribedTopicNames()); assertEquals("regex", member.subscribedTopicRegex()); assertEquals("range", member.serverAssignorName().get()); - assertEquals( - Collections.singletonList( - new ClientAssignor( - "assignor", - (byte) 0, - (byte) 0, - (byte) 1, - new VersionedMetadata( - (byte) 1, - ByteBuffer.allocate(0)))), - member.clientAssignors()); assertEquals(mkAssignment(mkTopicAssignment(topicId1, 1, 2, 3)), member.assignedPartitions()); assertEquals(mkAssignment(mkTopicAssignment(topicId2, 4, 5, 6)), member.partitionsPendingRevocation()); - assertEquals(mkAssignment(mkTopicAssignment(topicId3, 7, 8, 9)), member.partitionsPendingAssignment()); } @Test @@ -114,7 +89,6 @@ public void testEquals() { ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member-id") .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(11) .setInstanceId("instance-id") .setRackId("rack-id") .setRebalanceTimeoutMs(5000) @@ -123,27 +97,15 @@ public void testEquals() { .setSubscribedTopicNames(Arrays.asList("foo", "bar")) .setSubscribedTopicRegex("regex") .setServerAssignorName("range") - .setClientAssignors(Collections.singletonList( - new ClientAssignor( - "assignor", - (byte) 0, - (byte) 0, - (byte) 1, - new VersionedMetadata( - (byte) 1, - ByteBuffer.allocate(0))))) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3))) .setPartitionsPendingRevocation(mkAssignment( mkTopicAssignment(topicId2, 4, 5, 6))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(topicId3, 7, 8, 9))) .build(); ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member-id") .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(11) .setInstanceId("instance-id") .setRackId("rack-id") .setRebalanceTimeoutMs(5000) @@ -152,21 +114,10 @@ public void testEquals() { .setSubscribedTopicNames(Arrays.asList("foo", "bar")) .setSubscribedTopicRegex("regex") .setServerAssignorName("range") - .setClientAssignors(Collections.singletonList( - new ClientAssignor( - "assignor", - (byte) 0, - (byte) 0, - (byte) 1, - new VersionedMetadata( - (byte) 1, - ByteBuffer.allocate(0))))) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3))) .setPartitionsPendingRevocation(mkAssignment( mkTopicAssignment(topicId2, 4, 5, 6))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(topicId3, 7, 8, 9))) .build(); assertEquals(member1, member2); @@ -181,7 +132,6 @@ public void testUpdateMember() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member-id") .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(11) .setInstanceId("instance-id") .setRackId("rack-id") .setRebalanceTimeoutMs(5000) @@ -190,21 +140,10 @@ public void testUpdateMember() { .setSubscribedTopicNames(Arrays.asList("foo", "bar")) .setSubscribedTopicRegex("regex") .setServerAssignorName("range") - .setClientAssignors(Collections.singletonList( - new ClientAssignor( - "assignor", - (byte) 0, - (byte) 0, - (byte) 1, - new VersionedMetadata( - (byte) 1, - ByteBuffer.allocate(0))))) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3))) .setPartitionsPendingRevocation(mkAssignment( mkTopicAssignment(topicId2, 4, 5, 6))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(topicId3, 7, 8, 9))) .build(); // This is a no-op. @@ -215,7 +154,6 @@ public void testUpdateMember() { .maybeUpdateSubscribedTopicNames(Optional.empty()) .maybeUpdateSubscribedTopicRegex(Optional.empty()) .maybeUpdateRebalanceTimeoutMs(OptionalInt.empty()) - .maybeUpdateClientAssignors(Optional.empty()) .build(); assertEquals(member, updatedMember); @@ -227,7 +165,6 @@ public void testUpdateMember() { .maybeUpdateSubscribedTopicNames(Optional.of(Arrays.asList("zar"))) .maybeUpdateSubscribedTopicRegex(Optional.of("new-regex")) .maybeUpdateRebalanceTimeoutMs(OptionalInt.of(6000)) - .maybeUpdateClientAssignors(Optional.of(Collections.emptyList())) .build(); assertEquals("new-instance-id", updatedMember.instanceId()); @@ -236,18 +173,11 @@ public void testUpdateMember() { assertEquals(Arrays.asList("zar"), updatedMember.subscribedTopicNames()); assertEquals("new-regex", updatedMember.subscribedTopicRegex()); assertEquals("new-assignor", updatedMember.serverAssignorName().get()); - assertEquals(Collections.emptyList(), updatedMember.clientAssignors()); } @Test public void testUpdateWithConsumerGroupMemberMetadataValue() { ConsumerGroupMemberMetadataValue record = new ConsumerGroupMemberMetadataValue() - .setAssignors(Collections.singletonList(new ConsumerGroupMemberMetadataValue.Assignor() - .setName("client") - .setMinimumVersion((short) 0) - .setMaximumVersion((short) 2) - .setVersion((short) 1) - .setMetadata(new byte[0]))) .setServerAssignor("range") .setClientId("client-id") .setClientHost("host-id") @@ -269,17 +199,6 @@ public void testUpdateWithConsumerGroupMemberMetadataValue() { assertEquals(Arrays.asList("bar", "foo"), member.subscribedTopicNames()); assertEquals("regex", member.subscribedTopicRegex()); assertEquals("range", member.serverAssignorName().get()); - assertEquals( - Collections.singletonList( - new ClientAssignor( - "client", - (byte) 0, - (byte) 0, - (byte) 2, - new VersionedMetadata( - (byte) 1, - ByteBuffer.allocate(0)))), - member.clientAssignors()); } @Test @@ -291,16 +210,12 @@ public void testUpdateWithConsumerGroupCurrentMemberAssignmentValue() { ConsumerGroupCurrentMemberAssignmentValue record = new ConsumerGroupCurrentMemberAssignmentValue() .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setTargetMemberEpoch(11) .setAssignedPartitions(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() .setTopicId(topicId1) .setPartitions(Arrays.asList(0, 1, 2)))) .setPartitionsPendingRevocation(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() .setTopicId(topicId2) - .setPartitions(Arrays.asList(3, 4, 5)))) - .setPartitionsPendingAssignment(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() - .setTopicId(topicId3) - .setPartitions(Arrays.asList(6, 7, 8)))); + .setPartitions(Arrays.asList(3, 4, 5)))); ConsumerGroupMember member = new ConsumerGroupMember.Builder("member-id") .updateWith(record) @@ -308,10 +223,8 @@ public void testUpdateWithConsumerGroupCurrentMemberAssignmentValue() { assertEquals(10, member.memberEpoch()); assertEquals(9, member.previousMemberEpoch()); - assertEquals(11, member.targetMemberEpoch()); assertEquals(mkAssignment(mkTopicAssignment(topicId1, 0, 1, 2)), member.assignedPartitions()); assertEquals(mkAssignment(mkTopicAssignment(topicId2, 3, 4, 5)), member.partitionsPendingRevocation()); - assertEquals(mkAssignment(mkTopicAssignment(topicId3, 6, 7, 8)), member.partitionsPendingAssignment()); } @Test @@ -331,16 +244,12 @@ public void testAsConsumerGroupDescribeMember() { ConsumerGroupCurrentMemberAssignmentValue record = new ConsumerGroupCurrentMemberAssignmentValue() .setMemberEpoch(epoch) .setPreviousMemberEpoch(epoch - 1) - .setTargetMemberEpoch(epoch + 1) .setAssignedPartitions(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() .setTopicId(topicId1) .setPartitions(assignedPartitions))) .setPartitionsPendingRevocation(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() .setTopicId(topicId2) - .setPartitions(Arrays.asList(3, 4, 5)))) - .setPartitionsPendingAssignment(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() - .setTopicId(topicId3) - .setPartitions(Arrays.asList(6, 7, 8)))); + .setPartitions(Arrays.asList(3, 4, 5)))); String memberId = Uuid.randomUuid().toString(); String clientId = "clientId"; String instanceId = "instanceId"; diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java index 7d90122c2d..1526152706 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java @@ -177,8 +177,6 @@ public void testUpdatingMemberUpdatesPartitionEpoch() { mkTopicAssignment(fooTopicId, 1, 2, 3))) .setPartitionsPendingRevocation(mkAssignment( mkTopicAssignment(barTopicId, 4, 5, 6))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(zarTopicId, 7, 8, 9))) .build(); consumerGroup.updateMember(member); @@ -199,8 +197,6 @@ public void testUpdatingMemberUpdatesPartitionEpoch() { mkTopicAssignment(barTopicId, 1, 2, 3))) .setPartitionsPendingRevocation(mkAssignment( mkTopicAssignment(zarTopicId, 4, 5, 6))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 7, 8, 9))) .build(); consumerGroup.updateMember(member); @@ -337,8 +333,6 @@ public void testDeletingMemberRemovesPartitionEpoch() { mkTopicAssignment(fooTopicId, 1, 2, 3))) .setPartitionsPendingRevocation(mkAssignment( mkTopicAssignment(barTopicId, 4, 5, 6))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(zarTopicId, 7, 8, 9))) .build(); consumerGroup.updateMember(member); @@ -373,27 +367,27 @@ public void testGroupState() { assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY, consumerGroup.state()); ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1") + .setState(MemberState.STABLE) .setMemberEpoch(1) .setPreviousMemberEpoch(0) - .setTargetMemberEpoch(1) .build(); consumerGroup.updateMember(member1); consumerGroup.setGroupEpoch(1); - assertEquals(ConsumerGroupMember.MemberState.STABLE, member1.state()); + assertEquals(MemberState.STABLE, member1.state()); assertEquals(ConsumerGroup.ConsumerGroupState.ASSIGNING, consumerGroup.state()); ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member2") + .setState(MemberState.STABLE) .setMemberEpoch(1) .setPreviousMemberEpoch(0) - .setTargetMemberEpoch(1) .build(); consumerGroup.updateMember(member2); consumerGroup.setGroupEpoch(2); - assertEquals(ConsumerGroupMember.MemberState.STABLE, member2.state()); + assertEquals(MemberState.STABLE, member2.state()); assertEquals(ConsumerGroup.ConsumerGroupState.ASSIGNING, consumerGroup.state()); consumerGroup.setTargetAssignmentEpoch(2); @@ -401,42 +395,37 @@ public void testGroupState() { assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state()); member1 = new ConsumerGroupMember.Builder(member1) + .setState(MemberState.STABLE) .setMemberEpoch(2) .setPreviousMemberEpoch(1) - .setTargetMemberEpoch(2) .build(); consumerGroup.updateMember(member1); - assertEquals(ConsumerGroupMember.MemberState.STABLE, member1.state()); + assertEquals(MemberState.STABLE, member1.state()); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state()); // Member 2 is not stable so the group stays in reconciling state. member2 = new ConsumerGroupMember.Builder(member2) + .setState(MemberState.UNREVOKED_PARTITIONS) .setMemberEpoch(2) .setPreviousMemberEpoch(1) - .setTargetMemberEpoch(2) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 0))) .build(); consumerGroup.updateMember(member2); - assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, member2.state()); + assertEquals(MemberState.UNREVOKED_PARTITIONS, member2.state()); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state()); member2 = new ConsumerGroupMember.Builder(member2) + .setState(MemberState.STABLE) .setMemberEpoch(2) .setPreviousMemberEpoch(1) - .setTargetMemberEpoch(2) - .setAssignedPartitions(mkAssignment( - mkTopicAssignment(fooTopicId, 0))) - .setPartitionsPendingAssignment(Collections.emptyMap()) .build(); consumerGroup.updateMember(member2); - assertEquals(ConsumerGroupMember.MemberState.STABLE, member2.state()); + assertEquals(MemberState.STABLE, member2.state()); assertEquals(ConsumerGroup.ConsumerGroupState.STABLE, consumerGroup.state()); consumerGroup.removeMember("member1"); @@ -882,7 +871,6 @@ public void testValidateDeleteGroup() { ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1") .setMemberEpoch(1) .setPreviousMemberEpoch(0) - .setTargetMemberEpoch(1) .build(); consumerGroup.updateMember(member1); @@ -1014,7 +1002,6 @@ public void testStateTransitionMetrics() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") .setMemberEpoch(1) .setPreviousMemberEpoch(0) - .setTargetMemberEpoch(1) .build(); consumerGroup.updateMember(member); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java index 037a6ccbcd..c1db6f2289 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java @@ -17,542 +17,522 @@ package org.apache.kafka.coordinator.group.consumer; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.FencedMemberEpochException; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; -import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Stream; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; public class CurrentAssignmentBuilderTest { @Test - public void testTransitionFromNewTargetToRevoke() { + public void testStableToStable() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(10) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3), mkTopicAssignment(topicId2, 4, 5, 6))) .build(); - assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state()); - - Assignment targetAssignment = new Assignment(mkAssignment( - mkTopicAssignment(topicId1, 3, 4, 5), - mkTopicAssignment(topicId2, 6, 7, 8) - )); - ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) - .withTargetAssignment(11, targetAssignment) + .withTargetAssignment(11, new Assignment(mkAssignment( + mkTopicAssignment(topicId1, 1, 2, 3), + mkTopicAssignment(topicId2, 4, 5, 6)))) .withCurrentPartitionEpoch((topicId, partitionId) -> 10) .build(); - assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state()); - assertEquals(10, updatedMember.previousMemberEpoch()); - assertEquals(10, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.targetMemberEpoch()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 3), - mkTopicAssignment(topicId2, 6) - ), updatedMember.assignedPartitions()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 1, 2), - mkTopicAssignment(topicId2, 4, 5) - ), updatedMember.partitionsPendingRevocation()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 4, 5), - mkTopicAssignment(topicId2, 7, 8) - ), updatedMember.partitionsPendingAssignment()); + assertEquals( + new ConsumerGroupMember.Builder("member") + .setState(MemberState.STABLE) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(topicId1, 1, 2, 3), + mkTopicAssignment(topicId2, 4, 5, 6))) + .build(), + updatedMember + ); } @Test - public void testTransitionFromNewTargetToAssigning() { + public void testStableToStableWithNewPartitions() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(10) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3), mkTopicAssignment(topicId2, 4, 5, 6))) .build(); - assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state()); - - Assignment targetAssignment = new Assignment(mkAssignment( - mkTopicAssignment(topicId1, 1, 2, 3, 4, 5), - mkTopicAssignment(topicId2, 4, 5, 6, 7, 8) - )); - ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) - .withTargetAssignment(11, targetAssignment) - .withCurrentPartitionEpoch((topicId, partitionId) -> 10) + .withTargetAssignment(11, new Assignment(mkAssignment( + mkTopicAssignment(topicId1, 1, 2, 3, 4), + mkTopicAssignment(topicId2, 4, 5, 6, 7)))) + .withCurrentPartitionEpoch((topicId, partitionId) -> -1) .build(); - assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state()); - assertEquals(10, updatedMember.previousMemberEpoch()); - assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.targetMemberEpoch()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 1, 2, 3), - mkTopicAssignment(topicId2, 4, 5, 6) - ), updatedMember.assignedPartitions()); - assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 4, 5), - mkTopicAssignment(topicId2, 7, 8) - ), updatedMember.partitionsPendingAssignment()); + assertEquals( + new ConsumerGroupMember.Builder("member") + .setState(MemberState.STABLE) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(topicId1, 1, 2, 3, 4), + mkTopicAssignment(topicId2, 4, 5, 6, 7))) + .build(), + updatedMember + ); } @Test - public void testTransitionFromNewTargetToStable() { + public void testStableToUnrevokedPartitions() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") + .setState(MemberState.STABLE) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(10) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3), mkTopicAssignment(topicId2, 4, 5, 6))) .build(); - assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state()); + ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(11, new Assignment(mkAssignment( + mkTopicAssignment(topicId1, 2, 3, 4), + mkTopicAssignment(topicId2, 5, 6, 7)))) + .withCurrentPartitionEpoch((topicId, partitionId) -> -1) + .build(); - Assignment targetAssignment = new Assignment(mkAssignment( - mkTopicAssignment(topicId1, 1, 2, 3), - mkTopicAssignment(topicId2, 4, 5, 6) - )); + assertEquals( + new ConsumerGroupMember.Builder("member") + .setState(MemberState.UNREVOKED_PARTITIONS) + .setMemberEpoch(10) + .setPreviousMemberEpoch(10) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6))) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(topicId1, 1), + mkTopicAssignment(topicId2, 4))) + .build(), + updatedMember + ); + } + + @Test + public void testStableToUnreleasedPartitions() { + Uuid topicId1 = Uuid.randomUuid(); + Uuid topicId2 = Uuid.randomUuid(); + + ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") + .setState(MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(10) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(topicId1, 1, 2, 3), + mkTopicAssignment(topicId2, 4, 5, 6))) + .build(); ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) - .withTargetAssignment(11, targetAssignment) + .withTargetAssignment(11, new Assignment(mkAssignment( + mkTopicAssignment(topicId1, 1, 2, 3, 4), + mkTopicAssignment(topicId2, 4, 5, 6, 7)))) .withCurrentPartitionEpoch((topicId, partitionId) -> 10) .build(); - assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state()); - assertEquals(10, updatedMember.previousMemberEpoch()); - assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.targetMemberEpoch()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 1, 2, 3), - mkTopicAssignment(topicId2, 4, 5, 6) - ), updatedMember.assignedPartitions()); - assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation()); - assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment()); - } - - private static Stream ownedTopicPartitionsArguments() { - return Stream.of( - // Field not set in the heartbeat request. - null, - // Owned partitions does not match the assigned partitions. - Collections.emptyList() - ).map(Arguments::of); + assertEquals( + new ConsumerGroupMember.Builder("member") + .setState(MemberState.UNRELEASED_PARTITIONS) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(topicId1, 1, 2, 3), + mkTopicAssignment(topicId2, 4, 5, 6))) + .build(), + updatedMember + ); } - @ParameterizedTest - @MethodSource("ownedTopicPartitionsArguments") - public void testTransitionFromRevokeToRevoke( - List ownedTopicPartitions - ) { + @Test + public void testUnrevokedPartitionsToStable() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") + .setState(MemberState.UNREVOKED_PARTITIONS) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( - mkTopicAssignment(topicId1, 3), - mkTopicAssignment(topicId2, 6))) + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6))) .setPartitionsPendingRevocation(mkAssignment( - mkTopicAssignment(topicId1, 1, 2), - mkTopicAssignment(topicId2, 4, 5))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(topicId1, 4, 5), - mkTopicAssignment(topicId2, 7, 8))) + mkTopicAssignment(topicId1, 1), + mkTopicAssignment(topicId2, 4))) .build(); - assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state()); - - Assignment targetAssignment = new Assignment(mkAssignment( - mkTopicAssignment(topicId1, 3, 4, 5), - mkTopicAssignment(topicId2, 6, 7, 8) - )); - ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) - .withTargetAssignment(11, targetAssignment) + .withTargetAssignment(11, new Assignment(mkAssignment( + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6)))) .withCurrentPartitionEpoch((topicId, partitionId) -> -1) - .withOwnedTopicPartitions(ownedTopicPartitions) + .withOwnedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(topicId1) + .setPartitions(Arrays.asList(2, 3)), + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(topicId2) + .setPartitions(Arrays.asList(5, 6)))) .build(); - assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state()); - assertEquals(10, updatedMember.previousMemberEpoch()); - assertEquals(10, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.targetMemberEpoch()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 3), - mkTopicAssignment(topicId2, 6) - ), updatedMember.assignedPartitions()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 1, 2), - mkTopicAssignment(topicId2, 4, 5) - ), updatedMember.partitionsPendingRevocation()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 4, 5), - mkTopicAssignment(topicId2, 7, 8) - ), updatedMember.partitionsPendingAssignment()); + assertEquals( + new ConsumerGroupMember.Builder("member") + .setState(MemberState.STABLE) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6))) + .build(), + updatedMember + ); } @Test - public void testTransitionFromRevokeToAssigning() { + public void testRemainsInUnrevokedPartitions() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") + .setState(MemberState.UNREVOKED_PARTITIONS) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( - mkTopicAssignment(topicId1, 3), - mkTopicAssignment(topicId2, 6))) + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6))) .setPartitionsPendingRevocation(mkAssignment( - mkTopicAssignment(topicId1, 1, 2), - mkTopicAssignment(topicId2, 4, 5))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(topicId1, 4, 5), - mkTopicAssignment(topicId2, 7, 8))) + mkTopicAssignment(topicId1, 1), + mkTopicAssignment(topicId2, 4))) .build(); - assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state()); - - Assignment targetAssignment = new Assignment(mkAssignment( - mkTopicAssignment(topicId1, 3, 4, 5), - mkTopicAssignment(topicId2, 6, 7, 8) - )); - - ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) - .withTargetAssignment(11, targetAssignment) - .withCurrentPartitionEpoch((topicId, partitionId) -> 10) - .withOwnedTopicPartitions(requestFromAssignment(mkAssignment( + CurrentAssignmentBuilder currentAssignmentBuilder = new CurrentAssignmentBuilder(member) + .withTargetAssignment(12, new Assignment(mkAssignment( mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6)))) - .build(); - - assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state()); - assertEquals(10, updatedMember.previousMemberEpoch()); - assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.targetMemberEpoch()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 3), - mkTopicAssignment(topicId2, 6) - ), updatedMember.assignedPartitions()); - assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 4, 5), - mkTopicAssignment(topicId2, 7, 8) - ), updatedMember.partitionsPendingAssignment()); + .withCurrentPartitionEpoch((topicId, partitionId) -> -1); + + assertEquals( + member, + currentAssignmentBuilder + .withOwnedTopicPartitions(null) + .build() + ); + + assertEquals( + member, + currentAssignmentBuilder + .withOwnedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(topicId1) + .setPartitions(Arrays.asList(1, 2, 3)), + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(topicId2) + .setPartitions(Arrays.asList(5, 6)))) + .build() + ); + + assertEquals( + member, + currentAssignmentBuilder + .withOwnedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(topicId1) + .setPartitions(Arrays.asList(2, 3)), + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(topicId2) + .setPartitions(Arrays.asList(4, 5, 6)))) + .build() + ); } @Test - public void testTransitionFromRevokeToStable() { + public void testUnrevokedPartitionsToUnrevokedPartitions() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") + .setState(MemberState.UNREVOKED_PARTITIONS) .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( - mkTopicAssignment(topicId1, 3), - mkTopicAssignment(topicId2, 6))) + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6))) .setPartitionsPendingRevocation(mkAssignment( - mkTopicAssignment(topicId1, 1, 2), - mkTopicAssignment(topicId2, 4, 5))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(topicId1, 4, 5), - mkTopicAssignment(topicId2, 7, 8))) + mkTopicAssignment(topicId1, 1), + mkTopicAssignment(topicId2, 4))) .build(); - assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state()); - - Assignment targetAssignment = new Assignment(mkAssignment( - mkTopicAssignment(topicId1, 3, 4, 5), - mkTopicAssignment(topicId2, 6, 7, 8) - )); - ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) - .withTargetAssignment(11, targetAssignment) - .withCurrentPartitionEpoch((topicId, partitionId) -> -1) - .withOwnedTopicPartitions(requestFromAssignment(mkAssignment( + .withTargetAssignment(12, new Assignment(mkAssignment( mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6)))) + .withCurrentPartitionEpoch((topicId, partitionId) -> -1) + .withOwnedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(topicId1) + .setPartitions(Arrays.asList(2, 3)), + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(topicId2) + .setPartitions(Arrays.asList(5, 6)))) .build(); - assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state()); - assertEquals(10, updatedMember.previousMemberEpoch()); - assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.targetMemberEpoch()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 3, 4, 5), - mkTopicAssignment(topicId2, 6, 7, 8) - ), updatedMember.assignedPartitions()); - assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation()); - assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment()); + assertEquals( + new ConsumerGroupMember.Builder("member") + .setState(MemberState.UNREVOKED_PARTITIONS) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(topicId1, 3), + mkTopicAssignment(topicId2, 6))) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(topicId1, 2), + mkTopicAssignment(topicId2, 5))) + .build(), + updatedMember + ); } @Test - public void testTransitionFromRevokeToStableWhenPartitionsPendingRevocationAreReassignedBeforeBeingRevoked() { + public void testUnrevokedPartitionsToUnreleasedPartitions() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") - .setMemberEpoch(10) + .setState(MemberState.UNREVOKED_PARTITIONS) + .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( - mkTopicAssignment(topicId1, 3), - mkTopicAssignment(topicId2, 6))) - .setPartitionsPendingRevocation(mkAssignment( - mkTopicAssignment(topicId1, 1, 2), - mkTopicAssignment(topicId2, 4, 5))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(topicId1, 4, 5), - mkTopicAssignment(topicId2, 7, 8))) + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6))) .build(); - assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state()); - - // A new target assignment is computed (epoch 12) before the partitions - // pending revocation are revoked by the member and those partitions - // have been reassigned to the member. In this case, the member - // can keep them a jump to epoch 12. - Assignment targetAssignment = new Assignment(mkAssignment( - mkTopicAssignment(topicId1, 1, 2, 3), - mkTopicAssignment(topicId2, 4, 5, 6) - )); - ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) - .withTargetAssignment(12, targetAssignment) - .withCurrentPartitionEpoch((topicId, partitionId) -> -1) + .withTargetAssignment(11, new Assignment(mkAssignment( + mkTopicAssignment(topicId1, 2, 3, 4), + mkTopicAssignment(topicId2, 5, 6, 7)))) + .withCurrentPartitionEpoch((topicId, partitionId) -> 10) + .withOwnedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(topicId1) + .setPartitions(Arrays.asList(2, 3)), + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(topicId2) + .setPartitions(Arrays.asList(5, 6)))) .build(); - assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state()); - assertEquals(10, updatedMember.previousMemberEpoch()); - assertEquals(12, updatedMember.memberEpoch()); - assertEquals(12, updatedMember.targetMemberEpoch()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 1, 2, 3), - mkTopicAssignment(topicId2, 4, 5, 6) - ), updatedMember.assignedPartitions()); - assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation()); - assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment()); + assertEquals( + new ConsumerGroupMember.Builder("member") + .setState(MemberState.UNRELEASED_PARTITIONS) + .setMemberEpoch(11) + .setPreviousMemberEpoch(11) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6))) + .build(), + updatedMember + ); } @Test - public void testTransitionFromAssigningToAssigning() { + public void testUnreleasedPartitionsToStable() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") - .setMemberEpoch(10) + .setState(MemberState.UNRELEASED_PARTITIONS) + .setMemberEpoch(11) .setPreviousMemberEpoch(11) - .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( - mkTopicAssignment(topicId1, 3), - mkTopicAssignment(topicId2, 6))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(topicId1, 4, 5), - mkTopicAssignment(topicId2, 7, 8))) + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6))) .build(); - assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, member.state()); - - Assignment targetAssignment = new Assignment(mkAssignment( - mkTopicAssignment(topicId1, 3, 4, 5), - mkTopicAssignment(topicId2, 6, 7, 8) - )); - ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) - .withTargetAssignment(11, targetAssignment) - .withCurrentPartitionEpoch((topicId, partitionId) -> { - if (topicId.equals(topicId1)) - return -1; - else - return 10; - }) + .withTargetAssignment(12, new Assignment(mkAssignment( + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6)))) + .withCurrentPartitionEpoch((topicId, partitionId) -> 10) .build(); - assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state()); - assertEquals(10, updatedMember.previousMemberEpoch()); - assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.targetMemberEpoch()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 3, 4, 5), - mkTopicAssignment(topicId2, 6) - ), updatedMember.assignedPartitions()); - assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId2, 7, 8) - ), updatedMember.partitionsPendingAssignment()); + assertEquals( + new ConsumerGroupMember.Builder("member") + .setState(MemberState.STABLE) + .setMemberEpoch(12) + .setPreviousMemberEpoch(11) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6))) + .build(), + updatedMember + ); } @Test - public void testTransitionFromAssigningToStable() { + public void testUnreleasedPartitionsToStableWithNewPartitions() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") - .setMemberEpoch(10) + .setState(MemberState.UNRELEASED_PARTITIONS) + .setMemberEpoch(11) .setPreviousMemberEpoch(11) - .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( - mkTopicAssignment(topicId1, 3), - mkTopicAssignment(topicId2, 6))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(topicId1, 4, 5), - mkTopicAssignment(topicId2, 7, 8))) + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6))) .build(); - assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, member.state()); - - Assignment targetAssignment = new Assignment(mkAssignment( - mkTopicAssignment(topicId1, 3, 4, 5), - mkTopicAssignment(topicId2, 6, 7, 8) - )); - ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) - .withTargetAssignment(11, targetAssignment) + .withTargetAssignment(11, new Assignment(mkAssignment( + mkTopicAssignment(topicId1, 2, 3, 4), + mkTopicAssignment(topicId2, 5, 6, 7)))) .withCurrentPartitionEpoch((topicId, partitionId) -> -1) .build(); - assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state()); - assertEquals(10, updatedMember.previousMemberEpoch()); - assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.targetMemberEpoch()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 3, 4, 5), - mkTopicAssignment(topicId2, 6, 7, 8) - ), updatedMember.assignedPartitions()); - assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation()); - assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment()); + assertEquals( + new ConsumerGroupMember.Builder("member") + .setState(MemberState.STABLE) + .setMemberEpoch(11) + .setPreviousMemberEpoch(11) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(topicId1, 2, 3, 4), + mkTopicAssignment(topicId2, 5, 6, 7))) + .build(), + updatedMember + ); } @Test - public void testTransitionFromStableToStable() { + public void testUnreleasedPartitionsToUnreleasedPartitions() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") + .setState(MemberState.UNRELEASED_PARTITIONS) .setMemberEpoch(11) .setPreviousMemberEpoch(11) - .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( - mkTopicAssignment(topicId1, 3, 4, 5), - mkTopicAssignment(topicId2, 6, 7, 8))) + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6))) + .build(); + + ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(11, new Assignment(mkAssignment( + mkTopicAssignment(topicId1, 2, 3, 4), + mkTopicAssignment(topicId2, 5, 6, 7)))) + .withCurrentPartitionEpoch((topicId, partitionId) -> 10) .build(); - assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state()); + assertEquals(member, updatedMember); + } - Assignment targetAssignment = new Assignment(mkAssignment( - mkTopicAssignment(topicId1, 3, 4, 5), - mkTopicAssignment(topicId2, 6, 7, 8) - )); + @Test + public void testUnreleasedPartitionsToUnrevokedPartitions() { + Uuid topicId1 = Uuid.randomUuid(); + Uuid topicId2 = Uuid.randomUuid(); + + ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") + .setState(MemberState.UNRELEASED_PARTITIONS) + .setMemberEpoch(11) + .setPreviousMemberEpoch(11) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(topicId1, 2, 3), + mkTopicAssignment(topicId2, 5, 6))) + .build(); ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) - .withTargetAssignment(11, targetAssignment) - .withCurrentPartitionEpoch((topicId, partitionId) -> -1) + .withTargetAssignment(12, new Assignment(mkAssignment( + mkTopicAssignment(topicId1, 3), + mkTopicAssignment(topicId2, 6)))) + .withCurrentPartitionEpoch((topicId, partitionId) -> 10) .build(); - assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state()); - assertEquals(11, updatedMember.previousMemberEpoch()); - assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.targetMemberEpoch()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 3, 4, 5), - mkTopicAssignment(topicId2, 6, 7, 8) - ), updatedMember.assignedPartitions()); - assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation()); - assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment()); + assertEquals( + new ConsumerGroupMember.Builder("member") + .setState(MemberState.UNREVOKED_PARTITIONS) + .setMemberEpoch(11) + .setPreviousMemberEpoch(11) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(topicId1, 3), + mkTopicAssignment(topicId2, 6))) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(topicId1, 2), + mkTopicAssignment(topicId2, 5))) + .build(), + updatedMember + ); } @Test - public void testNewTargetRestartReconciliation() { + public void testUnknownState() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") - .setMemberEpoch(10) - .setPreviousMemberEpoch(10) - .setTargetMemberEpoch(11) + .setState(MemberState.UNKNOWN) + .setMemberEpoch(11) + .setPreviousMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6))) .setPartitionsPendingRevocation(mkAssignment( - mkTopicAssignment(topicId1, 1, 2), - mkTopicAssignment(topicId2, 4, 5))) - .setPartitionsPendingAssignment(mkAssignment( - mkTopicAssignment(topicId1, 4, 5), - mkTopicAssignment(topicId2, 7, 8))) + mkTopicAssignment(topicId1, 2), + mkTopicAssignment(topicId2, 5))) .build(); - assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state()); - - Assignment targetAssignment = new Assignment(mkAssignment( - mkTopicAssignment(topicId1, 6, 7, 8), - mkTopicAssignment(topicId2, 9, 10, 11) - )); + // When the member is in an unknown state, the member is first to force + // a reset of the client side member state. + assertThrows(FencedMemberEpochException.class, () -> new CurrentAssignmentBuilder(member) + .withTargetAssignment(12, new Assignment(mkAssignment( + mkTopicAssignment(topicId1, 3), + mkTopicAssignment(topicId2, 6)))) + .withCurrentPartitionEpoch((topicId, partitionId) -> 10) + .build()); + // Then the member rejoins with no owned partitions. ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member) - .withTargetAssignment(12, targetAssignment) - .withCurrentPartitionEpoch((topicId, partitionId) -> -1) + .withTargetAssignment(12, new Assignment(mkAssignment( + mkTopicAssignment(topicId1, 3), + mkTopicAssignment(topicId2, 6)))) + .withCurrentPartitionEpoch((topicId, partitionId) -> 11) + .withOwnedTopicPartitions(Collections.emptyList()) .build(); - assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state()); - assertEquals(10, updatedMember.previousMemberEpoch()); - assertEquals(10, updatedMember.memberEpoch()); - assertEquals(12, updatedMember.targetMemberEpoch()); - assertEquals(Collections.emptyMap(), updatedMember.assignedPartitions()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 1, 2, 3), - mkTopicAssignment(topicId2, 4, 5, 6) - ), updatedMember.partitionsPendingRevocation()); - assertEquals(mkAssignment( - mkTopicAssignment(topicId1, 6, 7, 8), - mkTopicAssignment(topicId2, 9, 10, 11) - ), updatedMember.partitionsPendingAssignment()); - } - - private static List requestFromAssignment( - Map> assignment - ) { - List topicPartitions = new ArrayList<>(); - - assignment.forEach((topicId, partitions) -> { - ConsumerGroupHeartbeatRequestData.TopicPartitions topic = new ConsumerGroupHeartbeatRequestData.TopicPartitions() - .setTopicId(topicId) - .setPartitions(new ArrayList<>(partitions)); - topicPartitions.add(topic); - }); - - return topicPartitions; + assertEquals( + new ConsumerGroupMember.Builder("member") + .setState(MemberState.STABLE) + .setMemberEpoch(12) + .setPreviousMemberEpoch(11) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(topicId1, 3), + mkTopicAssignment(topicId2, 6))) + .build(), + updatedMember + ); } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java index af94fede50..2a73e6fb54 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java @@ -90,11 +90,7 @@ private void addGroupMember( staticMembers.put(instanceId, memberId); } members.put(memberId, memberBuilder.build()); - targetAssignment.put(memberId, new Assignment( - (byte) 0, - targetPartitions, - VersionedMetadata.EMPTY - )); + targetAssignment.put(memberId, new Assignment(targetPartitions)); } public Uuid addTopicMetadata( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/VersionedMetadataTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/VersionedMetadataTest.java deleted file mode 100644 index 10997d015c..0000000000 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/VersionedMetadataTest.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.coordinator.group.consumer; - -import org.junit.jupiter.api.Test; - -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; - -public class VersionedMetadataTest { - @Test - public void testAttributes() { - VersionedMetadata metadata = new VersionedMetadata( - (short) 1, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)) - ); - - assertEquals((short) 1, metadata.version()); - assertEquals(ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)), metadata.metadata()); - } - - @Test - public void testMetadataCannotBeNull() { - assertThrows(NullPointerException.class, () -> new VersionedMetadata((short) 1, null)); - } - - @Test - public void testEquals() { - VersionedMetadata metadata = new VersionedMetadata( - (short) 1, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)) - ); - - assertEquals(new VersionedMetadata( - (short) 1, - ByteBuffer.wrap("hello".getBytes(StandardCharsets.UTF_8)) - ), metadata); - - assertNotEquals(VersionedMetadata.EMPTY, metadata); - } -} diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShardTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShardTest.java index b16ff6c653..5696f2bca6 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShardTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShardTest.java @@ -214,7 +214,7 @@ public void testConsumerGroupStateTransitionMetrics() { // Set member2 to ASSIGNING state. new ConsumerGroupMember.Builder(member2) - .setPartitionsPendingAssignment(Collections.singletonMap(Uuid.ZERO_UUID, Collections.singleton(0))) + .setPartitionsPendingRevocation(Collections.singletonMap(Uuid.ZERO_UUID, Collections.singleton(0))) .build(); snapshotRegistry.getOrCreateSnapshot(4000); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java index ba1a340e8e..c8a1dc337c 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.NotCoordinatorException; import org.apache.kafka.common.errors.NotEnoughReplicasException; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.requests.TransactionResult; @@ -43,14 +44,15 @@ import org.mockito.ArgumentMatcher; import java.time.Duration; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Deque; import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Objects; import java.util.OptionalInt; -import java.util.Queue; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -86,13 +88,24 @@ public class CoordinatorRuntimeTest { private static final TopicPartition TP = new TopicPartition("__consumer_offsets", 0); private static final Duration DEFAULT_WRITE_TIMEOUT = Duration.ofMillis(5); + private static final short TXN_OFFSET_COMMIT_LATEST_VERSION = ApiKeys.TXN_OFFSET_COMMIT.latestVersion(); + /** * A CoordinatorEventProcessor that directly executes the operations. This is * useful in unit tests where execution in threads is not required. */ private static class DirectEventProcessor implements CoordinatorEventProcessor { @Override - public void enqueue(CoordinatorEvent event) throws RejectedExecutionException { + public void enqueueLast(CoordinatorEvent event) throws RejectedExecutionException { + try { + event.run(); + } catch (Throwable ex) { + event.complete(ex); + } + } + + @Override + public void enqueueFirst(CoordinatorEvent event) throws RejectedExecutionException { try { event.run(); } catch (Throwable ex) { @@ -109,11 +122,16 @@ public void close() throws Exception {} * when poll() is called. */ private static class ManualEventProcessor implements CoordinatorEventProcessor { - private Queue queue = new LinkedList<>(); + private Deque queue = new LinkedList<>(); @Override - public void enqueue(CoordinatorEvent event) throws RejectedExecutionException { - queue.add(event); + public void enqueueLast(CoordinatorEvent event) throws RejectedExecutionException { + queue.addLast(event); + } + + @Override + public void enqueueFirst(CoordinatorEvent event) throws RejectedExecutionException { + queue.addFirst(event); } public boolean poll() { @@ -503,12 +521,6 @@ public void testScheduleLoadingWithFailure() { // Verify that onUnloaded is called. verify(coordinator, times(1)).onUnloaded(); - - // Verify that the listener is deregistered. - verify(writer, times(1)).deregisterListener( - eq(TP), - any(PartitionWriter.Listener.class) - ); } @Test @@ -1145,6 +1157,63 @@ public void testScheduleWriteOpWhenWriteTimesOut() throws InterruptedException { assertFutureThrows(timedOutWrite, org.apache.kafka.common.errors.TimeoutException.class); } + @Test + public void testScheduleWriteAllOperation() throws ExecutionException, InterruptedException, TimeoutException { + MockTimer timer = new MockTimer(); + MockPartitionWriter writer = new MockPartitionWriter(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .build(); + + TopicPartition coordinator0 = new TopicPartition("__consumer_offsets", 0); + TopicPartition coordinator1 = new TopicPartition("__consumer_offsets", 1); + TopicPartition coordinator2 = new TopicPartition("__consumer_offsets", 2); + + // Load coordinators. + runtime.scheduleLoadOperation(coordinator0, 10); + runtime.scheduleLoadOperation(coordinator1, 10); + runtime.scheduleLoadOperation(coordinator2, 10); + + // Writes. + AtomicInteger cnt = new AtomicInteger(0); + List>> writes = runtime.scheduleWriteAllOperation("write", DEFAULT_WRITE_TIMEOUT, state -> { + int counter = cnt.getAndIncrement(); + return new CoordinatorResult<>( + Collections.singletonList("record#" + counter), + Collections.singletonList("response#" + counter) + ); + }); + + assertEquals(1L, runtime.contextOrThrow(coordinator0).coordinator.lastWrittenOffset()); + assertEquals(1L, runtime.contextOrThrow(coordinator1).coordinator.lastWrittenOffset()); + assertEquals(1L, runtime.contextOrThrow(coordinator2).coordinator.lastWrittenOffset()); + + assertEquals(Collections.singletonList(InMemoryPartitionWriter.LogEntry.value("record#0")), writer.entries(coordinator0)); + assertEquals(Collections.singletonList(InMemoryPartitionWriter.LogEntry.value("record#1")), writer.entries(coordinator1)); + assertEquals(Collections.singletonList(InMemoryPartitionWriter.LogEntry.value("record#2")), writer.entries(coordinator2)); + + // Commit. + writer.commit(coordinator0); + writer.commit(coordinator1); + writer.commit(coordinator2); + + // Verify. + assertEquals( + Arrays.asList("response#0", "response#1", "response#2"), + FutureUtils.combineFutures(writes, ArrayList::new, List::addAll).get(5, TimeUnit.SECONDS) + ); + } + @Test public void testScheduleTransactionalWriteOp() { MockTimer timer = new MockTimer(); @@ -1188,7 +1257,8 @@ public CoordinatorShardBuilder get() { TP, "transactional-id", 100L, - (short) 50 + (short) 50, + TXN_OFFSET_COMMIT_LATEST_VERSION )).thenReturn(CompletableFuture.completedFuture(guard)); // Schedule a transactional write. @@ -1199,7 +1269,8 @@ public CoordinatorShardBuilder get() { 100L, (short) 50, Duration.ofMillis(5000), - state -> new CoordinatorResult<>(Arrays.asList("record1", "record2"), "response") + state -> new CoordinatorResult<>(Arrays.asList("record1", "record2"), "response"), + TXN_OFFSET_COMMIT_LATEST_VERSION ); // Verify that the writer got the records with the correct @@ -1270,7 +1341,8 @@ public CoordinatorShardBuilder get() { TP, "transactional-id", 100L, - (short) 50 + (short) 50, + TXN_OFFSET_COMMIT_LATEST_VERSION )).thenReturn(FutureUtils.failedFuture(Errors.NOT_ENOUGH_REPLICAS.exception())); // Schedule a transactional write. @@ -1281,7 +1353,8 @@ public CoordinatorShardBuilder get() { 100L, (short) 50, Duration.ofMillis(5000), - state -> new CoordinatorResult<>(Arrays.asList("record1", "record2"), "response") + state -> new CoordinatorResult<>(Arrays.asList("record1", "record2"), "response"), + TXN_OFFSET_COMMIT_LATEST_VERSION ); // Verify that the future is failed with the expected exception. @@ -1333,7 +1406,8 @@ public void testScheduleTransactionCompletion(TransactionResult result) throws E 100L, (short) 5, DEFAULT_WRITE_TIMEOUT, - state -> new CoordinatorResult<>(Arrays.asList("record1", "record2"), "response1") + state -> new CoordinatorResult<>(Arrays.asList("record1", "record2"), "response1"), + TXN_OFFSET_COMMIT_LATEST_VERSION ); // Verify that the write is not committed yet. @@ -1498,7 +1572,9 @@ public void testScheduleTransactionCompletionWhenWriteFails() { 100L, (short) 5, DEFAULT_WRITE_TIMEOUT, - state -> new CoordinatorResult<>(Arrays.asList("record1", "record2"), "response1")); + state -> new CoordinatorResult<>(Arrays.asList("record1", "record2"), "response1"), + TXN_OFFSET_COMMIT_LATEST_VERSION + ); // Verify that the state has been updated. assertEquals(2L, ctx.coordinator.lastWrittenOffset()); @@ -1580,7 +1656,9 @@ public void replayEndTransactionMarker( 100L, (short) 5, DEFAULT_WRITE_TIMEOUT, - state -> new CoordinatorResult<>(Arrays.asList("record1", "record2"), "response1")); + state -> new CoordinatorResult<>(Arrays.asList("record1", "record2"), "response1"), + TXN_OFFSET_COMMIT_LATEST_VERSION + ); // Verify that the state has been updated. assertEquals(2L, ctx.coordinator.lastWrittenOffset()); @@ -1743,6 +1821,58 @@ public void testScheduleReadOpWhenOpsFails() { assertFutureThrows(read, IllegalArgumentException.class); } + @Test + public void testScheduleReadAllOp() throws ExecutionException, InterruptedException, TimeoutException { + MockTimer timer = new MockTimer(); + MockPartitionWriter writer = new MockPartitionWriter(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .build(); + + TopicPartition coordinator0 = new TopicPartition("__consumer_offsets", 0); + TopicPartition coordinator1 = new TopicPartition("__consumer_offsets", 1); + TopicPartition coordinator2 = new TopicPartition("__consumer_offsets", 2); + + // Loads the coordinators. + runtime.scheduleLoadOperation(coordinator0, 10); + runtime.scheduleLoadOperation(coordinator1, 10); + runtime.scheduleLoadOperation(coordinator2, 10); + + // Writes + runtime.scheduleWriteOperation("write#0", coordinator0, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(Collections.singletonList("record0"), "response0")); + runtime.scheduleWriteOperation("write#1", coordinator1, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(Collections.singletonList("record1"), "response1")); + runtime.scheduleWriteOperation("write#2", coordinator2, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(Collections.singletonList("record2"), "response2")); + + // Commit writes. + writer.commit(coordinator0); + writer.commit(coordinator1); + writer.commit(coordinator2); + + // Read. + List>> responses = runtime.scheduleReadAllOperation( + "read", + (state, offset) -> new ArrayList<>(state.records) + ); + + assertEquals( + Arrays.asList("record0", "record1", "record2"), + FutureUtils.combineFutures(responses, ArrayList::new, List::addAll).get(5, TimeUnit.SECONDS) + ); + } + @Test public void testClose() throws Exception { MockCoordinatorLoader loader = spy(new MockCoordinatorLoader()); @@ -2481,6 +2611,75 @@ public void testPartitionLoadGeneratesSnapshotAtHighWatermarkNoRecordsLoaded() { assertTrue(ctx.coordinator.snapshotRegistry().hasSnapshot(0L)); } + @Test + public void testHighWatermarkUpdate() { + MockTimer timer = new MockTimer(); + MockPartitionWriter writer = new MockPartitionWriter(); + ManualEventProcessor processor = new ManualEventProcessor(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(processor) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .build(); + + // Loads the coordinator. Poll once to execute the load operation and once + // to complete the load. + runtime.scheduleLoadOperation(TP, 10); + processor.poll(); + processor.poll(); + + // Write #1. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(Collections.singletonList("record1"), "response1") + ); + processor.poll(); + + // Write #2. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(Collections.singletonList("record2"), "response2") + ); + processor.poll(); + + // Records have been written to the log. + assertEquals(Arrays.asList( + InMemoryPartitionWriter.LogEntry.value("record1"), + InMemoryPartitionWriter.LogEntry.value("record2") + ), writer.entries(TP)); + + // There is no pending high watermark. + assertEquals(-1, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); + + // Commit the first record. + writer.commit(TP, 1); + + // We should have one pending event and the pending high watermark should be set. + assertEquals(1, processor.size()); + assertEquals(1, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); + + // Commit the second record. + writer.commit(TP, 2); + + // We should still have one pending event and the pending high watermark should be updated. + assertEquals(1, processor.size()); + assertEquals(2, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); + + // Poll once to process the high watermark update and complete the writes. + processor.poll(); + + assertEquals(-1, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); + assertEquals(2, runtime.contextOrThrow(TP).coordinator.lastCommittedOffset()); + assertTrue(write1.isDone()); + assertTrue(write2.isDone()); + } + private static , U> ArgumentMatcher> coordinatorMatcher( CoordinatorRuntime runtime, TopicPartition tp diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/EventAccumulatorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/EventAccumulatorTest.java index 147cf08121..faac0f46f6 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/EventAccumulatorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/EventAccumulatorTest.java @@ -16,7 +16,9 @@ */ package org.apache.kafka.coordinator.group.runtime; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -78,7 +80,7 @@ public void testBasicOperations() { EventAccumulator accumulator = new EventAccumulator<>(); assertEquals(0, accumulator.size()); - assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS)); + assertNull(accumulator.poll()); List events = Arrays.asList( new MockEvent(1, 0), @@ -92,25 +94,53 @@ public void testBasicOperations() { new MockEvent(3, 2) ); - events.forEach(accumulator::add); + events.forEach(accumulator::addLast); assertEquals(9, accumulator.size()); Set polledEvents = new HashSet<>(); for (int i = 0; i < events.size(); i++) { - MockEvent event = accumulator.poll(0, TimeUnit.MICROSECONDS); + MockEvent event = accumulator.poll(); assertNotNull(event); polledEvents.add(event); assertEquals(events.size() - 1 - i, accumulator.size()); accumulator.done(event); } - assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS)); + assertNull(accumulator.poll()); assertEquals(new HashSet<>(events), polledEvents); assertEquals(0, accumulator.size()); accumulator.close(); } + @Test + public void testAddFirst() { + EventAccumulator accumulator = new EventAccumulator<>(); + + List events = Arrays.asList( + new MockEvent(1, 0), + new MockEvent(1, 1), + new MockEvent(1, 2) + ); + + events.forEach(accumulator::addFirst); + assertEquals(3, accumulator.size()); + + List polledEvents = new ArrayList<>(3); + for (int i = 0; i < events.size(); i++) { + MockEvent event = accumulator.poll(); + assertNotNull(event); + polledEvents.add(event); + assertEquals(events.size() - 1 - i, accumulator.size()); + accumulator.done(event); + } + + Collections.reverse(events); + assertEquals(events, polledEvents); + + accumulator.close(); + } + @Test public void testKeyConcurrentAndOrderingGuarantees() { EventAccumulator accumulator = new EventAccumulator<>(); @@ -118,35 +148,35 @@ public void testKeyConcurrentAndOrderingGuarantees() { MockEvent event0 = new MockEvent(1, 0); MockEvent event1 = new MockEvent(1, 1); MockEvent event2 = new MockEvent(1, 2); - accumulator.add(event0); - accumulator.add(event1); - accumulator.add(event2); + accumulator.addLast(event0); + accumulator.addLast(event1); + accumulator.addLast(event2); assertEquals(3, accumulator.size()); MockEvent event = null; // Poll event0. - event = accumulator.poll(0, TimeUnit.MICROSECONDS); + event = accumulator.poll(); assertEquals(event0, event); // Poll returns null because key is inflight. - assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS)); + assertNull(accumulator.poll()); accumulator.done(event); // Poll event1. - event = accumulator.poll(0, TimeUnit.MICROSECONDS); + event = accumulator.poll(); assertEquals(event1, event); // Poll returns null because key is inflight. - assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS)); + assertNull(accumulator.poll()); accumulator.done(event); // Poll event2. - event = accumulator.poll(0, TimeUnit.MICROSECONDS); + event = accumulator.poll(); assertEquals(event2, event); // Poll returns null because key is inflight. - assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS)); + assertNull(accumulator.poll()); accumulator.done(event); accumulator.close(); @@ -160,18 +190,18 @@ public void testDoneUnblockWaitingThreads() throws ExecutionException, Interrupt MockEvent event1 = new MockEvent(1, 1); MockEvent event2 = new MockEvent(1, 2); - CompletableFuture future0 = CompletableFuture.supplyAsync(accumulator::poll); - CompletableFuture future1 = CompletableFuture.supplyAsync(accumulator::poll); - CompletableFuture future2 = CompletableFuture.supplyAsync(accumulator::poll); + CompletableFuture future0 = CompletableFuture.supplyAsync(accumulator::take); + CompletableFuture future1 = CompletableFuture.supplyAsync(accumulator::take); + CompletableFuture future2 = CompletableFuture.supplyAsync(accumulator::take); List> futures = Arrays.asList(future0, future1, future2); assertFalse(future0.isDone()); assertFalse(future1.isDone()); assertFalse(future2.isDone()); - accumulator.add(event0); - accumulator.add(event1); - accumulator.add(event2); + accumulator.addLast(event0); + accumulator.addLast(event1); + accumulator.addLast(event2); // One future should be completed with event0. assertEquals(event0, CompletableFuture @@ -215,9 +245,9 @@ public void testDoneUnblockWaitingThreads() throws ExecutionException, Interrupt public void testCloseUnblockWaitingThreads() throws ExecutionException, InterruptedException, TimeoutException { EventAccumulator accumulator = new EventAccumulator<>(); - CompletableFuture future0 = CompletableFuture.supplyAsync(accumulator::poll); - CompletableFuture future1 = CompletableFuture.supplyAsync(accumulator::poll); - CompletableFuture future2 = CompletableFuture.supplyAsync(accumulator::poll); + CompletableFuture future0 = CompletableFuture.supplyAsync(accumulator::take); + CompletableFuture future1 = CompletableFuture.supplyAsync(accumulator::take); + CompletableFuture future2 = CompletableFuture.supplyAsync(accumulator::take); assertFalse(future0.isDone()); assertFalse(future1.isDone()); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/InMemoryPartitionWriter.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/InMemoryPartitionWriter.java index 7dea0b83f7..da6d662b6d 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/InMemoryPartitionWriter.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/InMemoryPartitionWriter.java @@ -274,7 +274,8 @@ public CompletableFuture maybeStartTransactionVerification( TopicPartition tp, String transactionalId, long producerId, - short producerEpoch + short producerEpoch, + short apiVersion ) throws KafkaException { return CompletableFuture.completedFuture(new VerificationGuard()); } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java index 2714188f65..f01fc88310 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java @@ -27,15 +27,12 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.LinkedList; import java.util.List; -import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -53,53 +50,19 @@ @Timeout(value = 60) public class MultiThreadedEventProcessorTest { - private static class MockEventAccumulator extends EventAccumulator { + private static class DelayEventAccumulator extends EventAccumulator { private final Time time; - private final Queue events; - private final long timeToPollMs; - private final AtomicBoolean isClosed; + private final long takeDelayMs; - public MockEventAccumulator(Time time, long timeToPollMs) { + public DelayEventAccumulator(Time time, long takeDelayMs) { this.time = time; - this.events = new LinkedList<>(); - this.timeToPollMs = timeToPollMs; - this.isClosed = new AtomicBoolean(false); + this.takeDelayMs = takeDelayMs; } @Override - public CoordinatorEvent poll() { - synchronized (events) { - while (events.isEmpty() && !isClosed.get()) { - try { - events.wait(); - } catch (Exception ignored) { - - } - } - time.sleep(timeToPollMs); - return events.poll(); - } - } - - @Override - public CoordinatorEvent poll(long timeout, TimeUnit unit) { - return null; - } - - @Override - public void add(CoordinatorEvent event) throws RejectedExecutionException { - synchronized (events) { - events.add(event); - events.notifyAll(); - } - } - - @Override - public void close() { - isClosed.set(true); - synchronized (events) { - events.notifyAll(); - } + public CoordinatorEvent take() { + time.sleep(takeDelayMs); + return super.take(); } } @@ -223,7 +186,7 @@ public void testEventsAreProcessed() throws Exception { new FutureEvent<>(new TopicPartition("foo", 2), numEventsExecuted::incrementAndGet) ); - events.forEach(eventProcessor::enqueue); + events.forEach(eventProcessor::enqueueLast); CompletableFuture.allOf(events .stream() @@ -260,7 +223,7 @@ public void testProcessingGuarantees() throws Exception { new FutureEvent<>(new TopicPartition("foo", 1), numEventsExecuted::incrementAndGet, true) // Event 5 ); - events.forEach(eventProcessor::enqueue); + events.forEach(eventProcessor::enqueueLast); // Events 0 and 1 are executed. assertTrue(events.get(0).awaitExecution(5, TimeUnit.SECONDS)); @@ -338,7 +301,7 @@ public void testEventsAreRejectedWhenClosed() throws Exception { eventProcessor.close(); assertThrows(RejectedExecutionException.class, - () -> eventProcessor.enqueue(new FutureEvent<>(new TopicPartition("foo", 0), () -> 0))); + () -> eventProcessor.enqueueLast(new FutureEvent<>(new TopicPartition("foo", 0), () -> 0))); } @Test @@ -353,7 +316,11 @@ public void testEventsAreDrainedWhenClosed() throws Exception { AtomicInteger numEventsExecuted = new AtomicInteger(0); // Special event which blocks until the latch is released. - FutureEvent blockingEvent = new FutureEvent<>(new TopicPartition("foo", 0), numEventsExecuted::incrementAndGet, true); + FutureEvent blockingEvent = new FutureEvent<>( + new TopicPartition("foo", 0), + numEventsExecuted::incrementAndGet, + true + ); List> events = Arrays.asList( new FutureEvent<>(new TopicPartition("foo", 0), numEventsExecuted::incrementAndGet), @@ -365,14 +332,14 @@ public void testEventsAreDrainedWhenClosed() throws Exception { ); // Enqueue the blocking event. - eventProcessor.enqueue(blockingEvent); + eventProcessor.enqueueLast(blockingEvent); // Ensure that the blocking event is executed. waitForCondition(() -> numEventsExecuted.get() > 0, "Blocking event not executed."); // Enqueue the other events. - events.forEach(eventProcessor::enqueue); + events.forEach(eventProcessor::enqueueLast); // Events should not be completed. events.forEach(event -> assertFalse(event.future.isDone())); @@ -382,7 +349,7 @@ public void testEventsAreDrainedWhenClosed() throws Exception { // Enqueuing a new event is rejected. assertThrows(RejectedExecutionException.class, - () -> eventProcessor.enqueue(blockingEvent)); + () -> eventProcessor.enqueueLast(blockingEvent)); // Release the blocking event to unblock the thread. blockingEvent.release(); @@ -428,10 +395,10 @@ public void testMetrics() throws Exception { 1, // Use a single thread to block event in the processor. mockTime, mockRuntimeMetrics, - new MockEventAccumulator<>(mockTime, 500L) + new DelayEventAccumulator(mockTime, 500L) )) { // Enqueue the blocking event. - eventProcessor.enqueue(blockingEvent); + eventProcessor.enqueueLast(blockingEvent); // Ensure that the blocking event is executed. waitForCondition(() -> numEventsExecuted.get() > 0, @@ -447,7 +414,7 @@ public void testMetrics() throws Exception { mockTime.milliseconds() ); - eventProcessor.enqueue(otherEvent); + eventProcessor.enqueueLast(otherEvent); // Pass the time. mockTime.sleep(3000L); @@ -501,7 +468,7 @@ public void testRecordThreadIdleRatioTwoThreads() throws Exception { 2, Time.SYSTEM, mockRuntimeMetrics, - new MockEventAccumulator<>(Time.SYSTEM, 100L) + new DelayEventAccumulator(Time.SYSTEM, 100L) )) { List recordedRatios = new ArrayList<>(); AtomicInteger numEventsExecuted = new AtomicInteger(0); @@ -525,7 +492,7 @@ public void testRecordThreadIdleRatioTwoThreads() throws Exception { new FutureEvent<>(new TopicPartition("foo", 2), numEventsExecuted::incrementAndGet) ); - events.forEach(eventProcessor::enqueue); + events.forEach(eventProcessor::enqueueLast); CompletableFuture.allOf(events .stream() diff --git a/metadata/src/main/java/org/apache/kafka/image/ConfigurationsDelta.java b/metadata/src/main/java/org/apache/kafka/image/ConfigurationsDelta.java index eab0505e33..9d794bfddc 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ConfigurationsDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/ConfigurationsDelta.java @@ -71,11 +71,12 @@ public void replay(ConfigRecord record) { public void replay(RemoveTopicRecord record, String topicName) { ConfigResource resource = new ConfigResource(Type.TOPIC, topicName); - ConfigurationImage configImage = image.resourceData().getOrDefault(resource, - new ConfigurationImage(resource, Collections.emptyMap())); - ConfigurationDelta delta = changes.computeIfAbsent(resource, - __ -> new ConfigurationDelta(configImage)); - delta.deleteAll(); + if (image.resourceData().containsKey(resource)) { + ConfigurationImage configImage = image.resourceData().get(resource); + ConfigurationDelta delta = changes.computeIfAbsent(resource, + __ -> new ConfigurationDelta(configImage)); + delta.deleteAll(); + } } public ConfigurationsImage apply() { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationZkWriter.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationZkWriter.java index 8a7e148d57..6870ad8f0c 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationZkWriter.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationZkWriter.java @@ -250,11 +250,6 @@ public void visitPartition(TopicIdPartition topicIdPartition, PartitionRegistrat migrationState -> migrationClient.topicClient().deleteTopic(topicName, migrationState) ); ConfigResource resource = new ConfigResource(ConfigResource.Type.TOPIC, topicName); - operationConsumer.accept( - UPDATE_TOPIC_CONFIG, - "Updating Configs for Topic " + topicName + ", ID " + topicId, - migrationState -> migrationClient.configClient().deleteConfigs(resource, migrationState) - ); }); newPartitions.forEach((topicId, partitionMap) -> { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java b/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java index 177d5311af..a7012d1505 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java @@ -17,7 +17,6 @@ package org.apache.kafka.metadata.placement; -import org.apache.kafka.common.DirectoryId; import org.apache.kafka.common.Uuid; import java.util.ArrayList; @@ -39,11 +38,6 @@ public class PartitionAssignment { private final List replicas; private final List directories; - // TODO remove -- just here for testing - public PartitionAssignment(List replicas) { - this(replicas, brokerId -> DirectoryId.UNASSIGNED); - } - public PartitionAssignment(List replicas, DefaultDirProvider defaultDirProvider) { this.replicas = Collections.unmodifiableList(new ArrayList<>(replicas)); Uuid[] directories = new Uuid[replicas.size()]; diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java index efc9bd2a24..b956a7f2ef 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java @@ -27,7 +27,6 @@ import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.metadata.Replicas; import org.apache.kafka.metadata.placement.DefaultDirProvider; -import org.apache.kafka.metadata.placement.PartitionAssignment; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Test; @@ -49,6 +48,7 @@ import static org.apache.kafka.controller.PartitionChangeBuilder.changeRecordIsNoOp; import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER; import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER_CHANGE; +import static org.apache.kafka.metadata.placement.PartitionAssignmentTest.partitionAssignment; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -502,7 +502,7 @@ public void testRevertReassignment(short version) { @MethodSource("partitionChangeRecordVersions") public void testRemovingReplicaReassignment(short version) { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - new PartitionAssignment(Replicas.toList(FOO.replicas)), new PartitionAssignment(Arrays.asList(1, 2))); + partitionAssignment(Replicas.toList(FOO.replicas)), partitionAssignment(Arrays.asList(1, 2))); assertEquals(Collections.singletonList(3), replicas.removing()); assertEquals(Collections.emptyList(), replicas.adding()); assertEquals(Arrays.asList(1, 2, 3), replicas.replicas()); @@ -527,7 +527,7 @@ public void testRemovingReplicaReassignment(short version) { @MethodSource("partitionChangeRecordVersions") public void testAddingReplicaReassignment(short version) { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - new PartitionAssignment(Replicas.toList(FOO.replicas)), new PartitionAssignment(Arrays.asList(1, 2, 3, 4))); + partitionAssignment(Replicas.toList(FOO.replicas)), partitionAssignment(Arrays.asList(1, 2, 3, 4))); assertEquals(Collections.emptyList(), replicas.removing()); assertEquals(Collections.singletonList(4), replicas.adding()); assertEquals(Arrays.asList(1, 2, 3, 4), replicas.replicas()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java index b2bc540bda..17be98d47f 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java @@ -24,10 +24,10 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.metadata.LeaderRecoveryState; import org.apache.kafka.metadata.PartitionRegistration; -import org.apache.kafka.metadata.placement.PartitionAssignment; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import static org.apache.kafka.metadata.placement.PartitionAssignmentTest.partitionAssignment; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -38,7 +38,7 @@ public class PartitionReassignmentReplicasTest { @Test public void testNoneAddedOrRemoved() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - new PartitionAssignment(Arrays.asList(3, 2, 1)), new PartitionAssignment(Arrays.asList(3, 2, 1))); + partitionAssignment(Arrays.asList(3, 2, 1)), partitionAssignment(Arrays.asList(3, 2, 1))); assertEquals(Collections.emptyList(), replicas.removing()); assertEquals(Collections.emptyList(), replicas.adding()); assertEquals(Arrays.asList(3, 2, 1), replicas.replicas()); @@ -47,7 +47,7 @@ public void testNoneAddedOrRemoved() { @Test public void testAdditions() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - new PartitionAssignment(Arrays.asList(3, 2, 1)), new PartitionAssignment(Arrays.asList(3, 6, 2, 1, 5))); + partitionAssignment(Arrays.asList(3, 2, 1)), partitionAssignment(Arrays.asList(3, 6, 2, 1, 5))); assertEquals(Collections.emptyList(), replicas.removing()); assertEquals(Arrays.asList(5, 6), replicas.adding()); assertEquals(Arrays.asList(3, 6, 2, 1, 5), replicas.replicas()); @@ -56,7 +56,7 @@ public void testAdditions() { @Test public void testRemovals() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - new PartitionAssignment(Arrays.asList(3, 2, 1, 0)), new PartitionAssignment(Arrays.asList(3, 1))); + partitionAssignment(Arrays.asList(3, 2, 1, 0)), partitionAssignment(Arrays.asList(3, 1))); assertEquals(Arrays.asList(0, 2), replicas.removing()); assertEquals(Collections.emptyList(), replicas.adding()); assertEquals(Arrays.asList(3, 1, 0, 2), replicas.replicas()); @@ -65,7 +65,7 @@ public void testRemovals() { @Test public void testAdditionsAndRemovals() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - new PartitionAssignment(Arrays.asList(3, 2, 1, 0)), new PartitionAssignment(Arrays.asList(7, 3, 1, 9))); + partitionAssignment(Arrays.asList(3, 2, 1, 0)), partitionAssignment(Arrays.asList(7, 3, 1, 9))); assertEquals(Arrays.asList(0, 2), replicas.removing()); assertEquals(Arrays.asList(7, 9), replicas.adding()); assertEquals(Arrays.asList(7, 3, 1, 9, 0, 2), replicas.replicas()); @@ -74,7 +74,7 @@ public void testAdditionsAndRemovals() { @Test public void testRearrangement() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - new PartitionAssignment(Arrays.asList(3, 2, 1, 0)), new PartitionAssignment(Arrays.asList(0, 1, 3, 2))); + partitionAssignment(Arrays.asList(3, 2, 1, 0)), partitionAssignment(Arrays.asList(0, 1, 3, 2))); assertEquals(Collections.emptyList(), replicas.removing()); assertEquals(Collections.emptyList(), replicas.adding()); assertEquals(Arrays.asList(0, 1, 3, 2), replicas.replicas()); @@ -83,7 +83,7 @@ public void testRearrangement() { @Test public void testDoesNotCompleteReassignment() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - new PartitionAssignment(Arrays.asList(0, 1, 2)), new PartitionAssignment(Arrays.asList(3, 4, 5))); + partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(3, 4, 5))); assertTrue(replicas.isReassignmentInProgress()); Optional reassignmentOptional = replicas.maybeCompleteReassignment(Arrays.asList(0, 1, 2, 3, 4)); @@ -107,7 +107,7 @@ public void testDoesNotCompleteReassignmentIfNoneOngoing() { @Test public void testDoesCompleteReassignmentAllNewReplicas() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - new PartitionAssignment(Arrays.asList(0, 1, 2)), new PartitionAssignment(Arrays.asList(3, 4, 5))); + partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(3, 4, 5))); assertTrue(replicas.isReassignmentInProgress()); Optional reassignmentOptional = replicas.maybeCompleteReassignment(Arrays.asList(0, 1, 2, 3, 4, 5)); @@ -120,7 +120,7 @@ public void testDoesCompleteReassignmentAllNewReplicas() { @Test public void testDoesCompleteReassignmentSomeNewReplicas() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - new PartitionAssignment(Arrays.asList(0, 1, 2)), new PartitionAssignment(Arrays.asList(0, 1, 3))); + partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(0, 1, 3))); assertTrue(replicas.isReassignmentInProgress()); Optional reassignmentOptional = replicas.maybeCompleteReassignment(Arrays.asList(0, 1, 2, 3)); @@ -199,7 +199,7 @@ public void testIsReassignmentInProgress() { @Test public void testDoesNotCompleteReassignmentIfIsrDoesNotHaveAllTargetReplicas() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - new PartitionAssignment(Arrays.asList(0, 1, 2)), new PartitionAssignment(Arrays.asList(0, 1, 3))); + partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(0, 1, 3))); assertTrue(replicas.isReassignmentInProgress()); Optional reassignmentOptional = replicas.maybeCompleteReassignment(Arrays.asList(3)); @@ -209,7 +209,7 @@ public void testDoesNotCompleteReassignmentIfIsrDoesNotHaveAllTargetReplicas() { @Test public void testOriginalReplicas() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - new PartitionAssignment(Arrays.asList(0, 1, 2)), new PartitionAssignment(Arrays.asList(0, 1, 3))); + partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(0, 1, 3))); assertEquals(Arrays.asList(0, 1, 2), replicas.originalReplicas()); } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index 3d54720be9..7e04818627 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -87,7 +87,6 @@ import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.metadata.Replicas; -import org.apache.kafka.metadata.placement.PartitionAssignment; import org.apache.kafka.metadata.placement.StripedReplicaPlacer; import org.apache.kafka.metadata.placement.UsableBroker; import org.apache.kafka.server.common.ApiMessageAndVersion; @@ -148,9 +147,11 @@ import static org.apache.kafka.controller.ControllerRequestContextUtil.anonymousContextFor; import static org.apache.kafka.controller.ControllerRequestContextUtil.anonymousContextWithMutationQuotaExceededFor; import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER; +import static org.apache.kafka.metadata.placement.PartitionAssignmentTest.partitionAssignment; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -323,7 +324,7 @@ void deleteTopic(ControllerRequestContext context, Uuid topicId) throws Exceptio assertEquals(1, result.records().size()); ApiMessageAndVersion removeRecordAndVersion = result.records().get(0); - assertTrue(removeRecordAndVersion.message() instanceof RemoveTopicRecord); + assertInstanceOf(RemoveTopicRecord.class, removeRecordAndVersion.message()); RemoveTopicRecord removeRecord = (RemoveTopicRecord) removeRecordAndVersion.message(); assertEquals(topicId, removeRecord.topicId()); @@ -1584,13 +1585,13 @@ public void testCreatePartitionsISRInvariants() throws Exception { public void testValidateGoodManualPartitionAssignments() throws Exception { ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder().build(); ctx.registerBrokers(1, 2, 3); - ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1)), OptionalInt.of(1)); - ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1)), OptionalInt.empty()); - ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1, 2, 3)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2, 3)), OptionalInt.of(3)); - ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1, 2, 3)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2, 3)), OptionalInt.empty()); } @@ -1600,20 +1601,20 @@ public void testValidateBadManualPartitionAssignments() throws Exception { ctx.registerBrokers(1, 2); assertEquals("The manual partition assignment includes an empty replica list.", assertThrows(InvalidReplicaAssignmentException.class, () -> - ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList()), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList()), OptionalInt.empty())).getMessage()); assertEquals("The manual partition assignment includes broker 3, but no such " + "broker is registered.", assertThrows(InvalidReplicaAssignmentException.class, () -> - ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1, 2, 3)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2, 3)), OptionalInt.empty())).getMessage()); assertEquals("The manual partition assignment includes the broker 2 more than " + "once.", assertThrows(InvalidReplicaAssignmentException.class, () -> - ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1, 2, 2)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2, 2)), OptionalInt.empty())).getMessage()); assertEquals("The manual partition assignment includes a partition with 2 " + "replica(s), but this is not consistent with previous partitions, which have " + "3 replica(s).", assertThrows(InvalidReplicaAssignmentException.class, () -> - ctx.replicationControl.validateManualPartitionAssignment(new PartitionAssignment(asList(1, 2)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2)), OptionalInt.of(3))).getMessage()); } @@ -2213,7 +2214,7 @@ public void testElectUncleanLeaders_WithoutElr(boolean electAllPartitions) throw assertEquals(1, result.records().size()); ApiMessageAndVersion record = result.records().get(0); - assertTrue(record.message() instanceof PartitionChangeRecord); + assertInstanceOf(PartitionChangeRecord.class, record.message()); PartitionChangeRecord partitionChangeRecord = (PartitionChangeRecord) record.message(); assertEquals(0, partitionChangeRecord.partitionId()); diff --git a/metadata/src/test/java/org/apache/kafka/image/ConfigurationsImageTest.java b/metadata/src/test/java/org/apache/kafka/image/ConfigurationsImageTest.java index 9b7cd39dcd..995a44b3f8 100644 --- a/metadata/src/test/java/org/apache/kafka/image/ConfigurationsImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/ConfigurationsImageTest.java @@ -27,6 +27,7 @@ import org.junit.jupiter.api.Timeout; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -61,26 +62,34 @@ public class ConfigurationsImageTest { IMAGE1 = new ConfigurationsImage(map1); DELTA1_RECORDS = new ArrayList<>(); + // remove configs DELTA1_RECORDS.add(new ApiMessageAndVersion(new ConfigRecord().setResourceType(BROKER.id()). setResourceName("0").setName("foo").setValue(null), CONFIG_RECORD.highestSupportedVersion())); + DELTA1_RECORDS.add(new ApiMessageAndVersion(new ConfigRecord().setResourceType(BROKER.id()). + setResourceName("0").setName("baz").setValue(null), + CONFIG_RECORD.highestSupportedVersion())); + DELTA1_RECORDS.add(new ApiMessageAndVersion(new ConfigRecord().setResourceType(BROKER.id()). + setResourceName("1").setName("foobar").setValue(null), + CONFIG_RECORD.highestSupportedVersion())); + // add new config to b1 DELTA1_RECORDS.add(new ApiMessageAndVersion(new ConfigRecord().setResourceType(BROKER.id()). setResourceName("1").setName("barfoo").setValue("bazfoo"), CONFIG_RECORD.highestSupportedVersion())); + // add new config to b2 + DELTA1_RECORDS.add(new ApiMessageAndVersion(new ConfigRecord().setResourceType(BROKER.id()). + setResourceName("2").setName("foo").setValue("bar"), + CONFIG_RECORD.highestSupportedVersion())); DELTA1 = new ConfigurationsDelta(IMAGE1); RecordTestUtils.replayAll(DELTA1, DELTA1_RECORDS); Map map2 = new HashMap<>(); - Map broker0Map2 = new HashMap<>(); - broker0Map2.put("baz", "quux"); - map2.put(new ConfigResource(BROKER, "0"), - new ConfigurationImage(new ConfigResource(BROKER, "0"), broker0Map2)); - Map broker1Map2 = new HashMap<>(); - broker1Map2.put("foobar", "foobaz"); - broker1Map2.put("barfoo", "bazfoo"); + Map broker1Map2 = Collections.singletonMap("barfoo", "bazfoo"); map2.put(new ConfigResource(BROKER, "1"), new ConfigurationImage(new ConfigResource(BROKER, "1"), broker1Map2)); + Map broker2Map = Collections.singletonMap("foo", "bar"); + map2.put(new ConfigResource(BROKER, "2"), new ConfigurationImage(new ConfigResource(BROKER, "2"), broker2Map)); IMAGE2 = new ConfigurationsImage(map2); } diff --git a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java index 84a092ccd3..eabb63ff85 100644 --- a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java @@ -94,10 +94,16 @@ private static ImmutableMap newTopicsByNameMap(Collection newTopicsByNameMap(Collection(); + // remove topic DELTA1_RECORDS.add(new ApiMessageAndVersion(new RemoveTopicRecord(). setTopicId(FOO_UUID), REMOVE_TOPIC_RECORD.highestSupportedVersion())); + // change topic DELTA1_RECORDS.add(new ApiMessageAndVersion(new PartitionChangeRecord(). setTopicId(BAR_UUID). setPartitionId(0).setLeader(1), PARTITION_CHANGE_RECORD.highestSupportedVersion())); + // add topic DELTA1_RECORDS.add(new ApiMessageAndVersion(new TopicRecord(). setName("baz").setTopicId(BAZ_UUID), TOPIC_RECORD.highestSupportedVersion())); + // add partition record for new topic DELTA1_RECORDS.add(new ApiMessageAndVersion(new PartitionRecord(). setPartitionId(0). setTopicId(BAZ_UUID). @@ -138,11 +148,23 @@ private static ImmutableMap newTopicsByNameMap(Collection topics2 = Arrays.asList( + newTopicImage("foo", FOO_UUID2), newTopicImage("bar", BAR_UUID, new PartitionRegistration.Builder().setReplicas(new int[] {0, 1, 2, 3, 4}). setDirectories(DirectoryId.migratingArray(5)). @@ -188,22 +210,22 @@ private PartitionRegistration newPartition(int[] replicas) { public void testBasicLocalChanges() { int localId = 3; /* Changes already include in DELTA1_RECORDS and IMAGE1: - * foo - topic id deleted + * foo - topic id deleted then recreated with different topic id * bar-0 - stay as follower with different partition epoch * baz-0 - new topic to leader + * bam - topic id created then deleted */ List topicRecords = new ArrayList<>(DELTA1_RECORDS); - // Create a new foo topic with a different id - Uuid newFooId = Uuid.fromString("b66ybsWIQoygs01vdjH07A"); + // Create a new bam topic with a different id topicRecords.add( new ApiMessageAndVersion( - new TopicRecord().setName("foo") .setTopicId(newFooId), + new TopicRecord().setName("bam").setTopicId(BAM_UUID2), TOPIC_RECORD.highestSupportedVersion() ) ); - topicRecords.add(newPartitionRecord(newFooId, 0, Arrays.asList(0, 1, 2))); - topicRecords.add(newPartitionRecord(newFooId, 1, Arrays.asList(0, 1, localId))); + topicRecords.add(newPartitionRecord(BAM_UUID2, 0, Arrays.asList(0, 1, 2))); + topicRecords.add(newPartitionRecord(BAM_UUID2, 1, Arrays.asList(0, 1, localId))); // baz-1 - new partition to follower topicRecords.add( @@ -224,10 +246,6 @@ public void testBasicLocalChanges() { RecordTestUtils.replayAll(delta, topicRecords); LocalReplicaChanges changes = delta.localChanges(localId); - assertEquals( - new HashSet<>(Arrays.asList(new TopicPartition("foo", 0), new TopicPartition("foo", 1))), - changes.deletes() - ); assertEquals( new HashSet<>(Arrays.asList(new TopicPartition("baz", 0))), changes.electedLeaders().keySet() @@ -238,7 +256,8 @@ public void testBasicLocalChanges() { ); assertEquals( new HashSet<>( - Arrays.asList(new TopicPartition("baz", 1), new TopicPartition("bar", 0), new TopicPartition("foo", 1)) + Arrays.asList(new TopicPartition("baz", 1), new TopicPartition("bar", 0), + new TopicPartition("bam", 1)) ), changes.followers().keySet() ); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/migration/KRaftMigrationDriverTest.java b/metadata/src/test/java/org/apache/kafka/metadata/migration/KRaftMigrationDriverTest.java index 17ca7ab863..0d6af36c8f 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/migration/KRaftMigrationDriverTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/migration/KRaftMigrationDriverTest.java @@ -573,7 +573,7 @@ public void testTopicDualWriteSnapshot() throws Exception { TestUtils.waitForCondition(() -> driver.migrationState().get(1, TimeUnit.MINUTES).equals(MigrationDriverState.DUAL_WRITE), "Waiting for KRaftMigrationDriver to enter ZK_MIGRATION state"); - // Modify topics in a KRaft snapshot -- delete foo, modify bar, add baz + // Modify topics in a KRaft snapshot -- delete foo, modify bar, add baz, add new foo, add bam, delete bam provenance = new MetadataProvenance(200, 1, 1); delta = new MetadataDelta(image); RecordTestUtils.replayAll(delta, DELTA1_RECORDS); @@ -583,10 +583,11 @@ public void testTopicDualWriteSnapshot() throws Exception { assertEquals(1, topicClient.deletedTopics.size()); assertEquals("foo", topicClient.deletedTopics.get(0)); - assertEquals(1, topicClient.createdTopics.size()); - assertEquals("baz", topicClient.createdTopics.get(0)); + assertEquals(2, topicClient.createdTopics.size()); + assertTrue(topicClient.createdTopics.contains("foo")); + assertTrue(topicClient.createdTopics.contains("baz")); assertTrue(topicClient.updatedTopicPartitions.get("bar").contains(0)); - assertEquals(new ConfigResource(ConfigResource.Type.TOPIC, "foo"), configClient.deletedResources.get(0)); + assertEquals(0, configClient.deletedResources.size()); }); } @@ -630,7 +631,7 @@ public void testTopicDualWriteDelta() throws Exception { TestUtils.waitForCondition(() -> driver.migrationState().get(1, TimeUnit.MINUTES).equals(MigrationDriverState.DUAL_WRITE), "Waiting for KRaftMigrationDriver to enter DUAL_WRITE state"); - // Modify topics in a KRaft snapshot -- delete foo, modify bar, add baz + // Modify topics in a KRaft snapshot -- delete foo, modify bar, add baz, add new foo, add bam, delete bam provenance = new MetadataProvenance(200, 1, 1); delta = new MetadataDelta(image); RecordTestUtils.replayAll(delta, DELTA1_RECORDS); @@ -640,10 +641,11 @@ public void testTopicDualWriteDelta() throws Exception { assertEquals(1, topicClient.deletedTopics.size()); assertEquals("foo", topicClient.deletedTopics.get(0)); - assertEquals(1, topicClient.createdTopics.size()); - assertEquals("baz", topicClient.createdTopics.get(0)); + assertEquals(2, topicClient.createdTopics.size()); + assertTrue(topicClient.createdTopics.contains("foo")); + assertTrue(topicClient.createdTopics.contains("baz")); assertTrue(topicClient.updatedTopicPartitions.get("bar").contains(0)); - assertEquals(new ConfigResource(ConfigResource.Type.TOPIC, "foo"), configClient.deletedResources.get(0)); + assertEquals(0, configClient.deletedResources.size()); }); } @@ -745,7 +747,7 @@ public void testControllerFailover() throws Exception { migrationClient.setMigrationRecoveryState( ZkMigrationLeadershipState.EMPTY.withKRaftMetadataOffsetAndEpoch(100, 1)); - // Modify topics in a KRaft -- delete foo, modify bar, add baz + // Modify topics in a KRaft -- delete foo, modify bar, add baz, add new foo, add bam, delete bam provenance = new MetadataProvenance(200, 1, 1); delta = new MetadataDelta(image); RecordTestUtils.replayAll(delta, DELTA1_RECORDS); @@ -761,10 +763,11 @@ public void testControllerFailover() throws Exception { ""); assertEquals(1, topicClient.deletedTopics.size()); assertEquals("foo", topicClient.deletedTopics.get(0)); - assertEquals(1, topicClient.createdTopics.size()); - assertEquals("baz", topicClient.createdTopics.get(0)); + assertEquals(2, topicClient.createdTopics.size()); + assertTrue(topicClient.createdTopics.contains("foo")); + assertTrue(topicClient.createdTopics.contains("baz")); assertTrue(topicClient.updatedTopicPartitions.get("bar").contains(0)); - assertEquals(new ConfigResource(ConfigResource.Type.TOPIC, "foo"), configClient.deletedResources.get(0)); + assertEquals(0, configClient.deletedResources.size()); }); } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java b/metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java index 06cf5ae50d..6dca18b4dd 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java @@ -20,6 +20,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; +import org.apache.kafka.common.DirectoryId; import org.apache.kafka.common.Uuid; import org.junit.jupiter.api.Test; @@ -27,20 +28,23 @@ import java.util.List; public class PartitionAssignmentTest { + public static PartitionAssignment partitionAssignment(List replicas) { + return new PartitionAssignment(replicas, __ -> DirectoryId.MIGRATING); + } @Test public void testPartitionAssignmentReplicas() { List replicas = Arrays.asList(0, 1, 2); - assertEquals(replicas, new PartitionAssignment(replicas).replicas()); + assertEquals(replicas, partitionAssignment(replicas).replicas()); } @Test public void testConsistentEqualsAndHashCode() { List partitionAssignments = Arrays.asList( - new PartitionAssignment( + partitionAssignment( Arrays.asList(0, 1, 2) ), - new PartitionAssignment( + partitionAssignment( Arrays.asList(1, 2, 0) ) ); @@ -49,7 +53,7 @@ public void testConsistentEqualsAndHashCode() { for (int j = 0; j < partitionAssignments.size(); j++) { if (i == j) { assertEquals(partitionAssignments.get(i), partitionAssignments.get(j)); - assertEquals(partitionAssignments.get(i), new PartitionAssignment(partitionAssignments.get(i).replicas())); + assertEquals(partitionAssignments.get(i), partitionAssignment(partitionAssignments.get(i).replicas())); assertEquals(partitionAssignments.get(i).hashCode(), partitionAssignments.get(j).hashCode()); } else { assertNotEquals(partitionAssignments.get(i), partitionAssignments.get(j)); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java b/metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java index 8b02416d2b..924fcdb755 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java @@ -34,6 +34,7 @@ import java.util.Map; import java.util.Optional; +import static org.apache.kafka.metadata.placement.PartitionAssignmentTest.partitionAssignment; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -105,7 +106,7 @@ public Iterator usableBrokers() { @Override public Uuid defaultDir(int brokerId) { - return DirectoryId.UNASSIGNED; + return DirectoryId.MIGRATING; } }); } @@ -118,9 +119,9 @@ public Uuid defaultDir(int brokerId) { public void testMultiPartitionTopicPlacementOnSingleUnfencedBroker() { MockRandom random = new MockRandom(); StripedReplicaPlacer placer = new StripedReplicaPlacer(random); - assertEquals(new TopicAssignment(Arrays.asList(new PartitionAssignment(Arrays.asList(0)), - new PartitionAssignment(Arrays.asList(0)), - new PartitionAssignment(Arrays.asList(0)))), + assertEquals(new TopicAssignment(Arrays.asList(partitionAssignment(Arrays.asList(0)), + partitionAssignment(Arrays.asList(0)), + partitionAssignment(Arrays.asList(0)))), place(placer, 0, 3, (short) 1, Arrays.asList( new UsableBroker(0, Optional.empty(), false), new UsableBroker(1, Optional.empty(), true)))); @@ -224,11 +225,11 @@ public void testNonPositiveReplicationFactor() { public void testSuccessfulPlacement() { MockRandom random = new MockRandom(); StripedReplicaPlacer placer = new StripedReplicaPlacer(random); - assertEquals(new TopicAssignment(Arrays.asList(new PartitionAssignment(Arrays.asList(2, 3, 0)), - new PartitionAssignment(Arrays.asList(3, 0, 1)), - new PartitionAssignment(Arrays.asList(0, 1, 2)), - new PartitionAssignment(Arrays.asList(1, 2, 3)), - new PartitionAssignment(Arrays.asList(1, 0, 2)))), + assertEquals(new TopicAssignment(Arrays.asList(partitionAssignment(Arrays.asList(2, 3, 0)), + partitionAssignment(Arrays.asList(3, 0, 1)), + partitionAssignment(Arrays.asList(0, 1, 2)), + partitionAssignment(Arrays.asList(1, 2, 3)), + partitionAssignment(Arrays.asList(1, 0, 2)))), place(placer, 0, 5, (short) 3, Arrays.asList( new UsableBroker(0, Optional.empty(), false), new UsableBroker(3, Optional.empty(), false), diff --git a/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java b/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java index 7b5a24c3b8..26f8841d83 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.metadata.placement; +import static org.apache.kafka.metadata.placement.PartitionAssignmentTest.partitionAssignment; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -33,8 +34,8 @@ public void testTopicAssignmentReplicas() { List replicasP0 = Arrays.asList(0, 1, 2); List replicasP1 = Arrays.asList(1, 2, 0); List partitionAssignments = Arrays.asList( - new PartitionAssignment(replicasP0), - new PartitionAssignment(replicasP1) + partitionAssignment(replicasP0), + partitionAssignment(replicasP1) ); assertEquals(partitionAssignments, new TopicAssignment(partitionAssignments).assignments()); } @@ -44,14 +45,14 @@ public void testConsistentEqualsAndHashCode() { List topicAssignments = Arrays.asList( new TopicAssignment( Arrays.asList( - new PartitionAssignment( + partitionAssignment( Arrays.asList(0, 1, 2) ) ) ), new TopicAssignment( Arrays.asList( - new PartitionAssignment( + partitionAssignment( Arrays.asList(1, 2, 0) ) ) diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index 44d39880aa..07d8b5a0d6 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -52,6 +52,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -1938,7 +1939,7 @@ private static Optional assertFetchS int replicaId, int maxBytes ) { - assertTrue(request.data() instanceof FetchSnapshotRequestData); + assertInstanceOf(FetchSnapshotRequestData.class, request.data()); FetchSnapshotRequestData data = (FetchSnapshotRequestData) request.data(); diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index ea4078b7c5..97eb91dfb4 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -83,6 +83,7 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -452,8 +453,9 @@ DescribeQuorumResponseData collectDescribeQuorumResponse() { List sentMessages = drainSentResponses(ApiKeys.DESCRIBE_QUORUM); assertEquals(1, sentMessages.size()); RaftResponse.Outbound raftMessage = sentMessages.get(0); - assertTrue( - raftMessage.data() instanceof DescribeQuorumResponseData, + assertInstanceOf( + DescribeQuorumResponseData.class, + raftMessage.data(), "Unexpected request type " + raftMessage.data()); return (DescribeQuorumResponseData) raftMessage.data(); } @@ -493,7 +495,7 @@ void assertSentVoteResponse( List sentMessages = drainSentResponses(ApiKeys.VOTE); assertEquals(1, sentMessages.size()); RaftMessage raftMessage = sentMessages.get(0); - assertTrue(raftMessage.data() instanceof VoteResponseData); + assertInstanceOf(VoteResponseData.class, raftMessage.data()); VoteResponseData response = (VoteResponseData) raftMessage.data(); assertEquals(error, Errors.forCode(response.errorCode())); @@ -508,7 +510,7 @@ void assertSentVoteResponse( List sentMessages = drainSentResponses(ApiKeys.VOTE); assertEquals(1, sentMessages.size()); RaftMessage raftMessage = sentMessages.get(0); - assertTrue(raftMessage.data() instanceof VoteResponseData); + assertInstanceOf(VoteResponseData.class, raftMessage.data()); VoteResponseData response = (VoteResponseData) raftMessage.data(); assertTrue(hasValidTopicPartition(response, metadataPartition)); @@ -585,7 +587,7 @@ void assertSentBeginQuorumEpochResponse( List sentMessages = drainSentResponses(ApiKeys.BEGIN_QUORUM_EPOCH); assertEquals(1, sentMessages.size()); RaftMessage raftMessage = sentMessages.get(0); - assertTrue(raftMessage.data() instanceof BeginQuorumEpochResponseData); + assertInstanceOf(BeginQuorumEpochResponseData.class, raftMessage.data()); BeginQuorumEpochResponseData response = (BeginQuorumEpochResponseData) raftMessage.data(); assertEquals(responseError, Errors.forCode(response.errorCode())); } @@ -598,7 +600,7 @@ void assertSentBeginQuorumEpochResponse( List sentMessages = drainSentResponses(ApiKeys.BEGIN_QUORUM_EPOCH); assertEquals(1, sentMessages.size()); RaftMessage raftMessage = sentMessages.get(0); - assertTrue(raftMessage.data() instanceof BeginQuorumEpochResponseData); + assertInstanceOf(BeginQuorumEpochResponseData.class, raftMessage.data()); BeginQuorumEpochResponseData response = (BeginQuorumEpochResponseData) raftMessage.data(); assertEquals(Errors.NONE, Errors.forCode(response.errorCode())); @@ -623,7 +625,7 @@ void assertSentEndQuorumEpochResponse( List sentMessages = drainSentResponses(ApiKeys.END_QUORUM_EPOCH); assertEquals(1, sentMessages.size()); RaftMessage raftMessage = sentMessages.get(0); - assertTrue(raftMessage.data() instanceof EndQuorumEpochResponseData); + assertInstanceOf(EndQuorumEpochResponseData.class, raftMessage.data()); EndQuorumEpochResponseData response = (EndQuorumEpochResponseData) raftMessage.data(); assertEquals(responseError, Errors.forCode(response.errorCode())); } @@ -636,7 +638,7 @@ void assertSentEndQuorumEpochResponse( List sentMessages = drainSentResponses(ApiKeys.END_QUORUM_EPOCH); assertEquals(1, sentMessages.size()); RaftMessage raftMessage = sentMessages.get(0); - assertTrue(raftMessage.data() instanceof EndQuorumEpochResponseData); + assertInstanceOf(EndQuorumEpochResponseData.class, raftMessage.data()); EndQuorumEpochResponseData response = (EndQuorumEpochResponseData) raftMessage.data(); assertEquals(Errors.NONE, Errors.forCode(response.errorCode())); @@ -736,7 +738,7 @@ void assertSentFetchSnapshotResponse(Errors responseError) { assertEquals(1, sentMessages.size()); RaftMessage message = sentMessages.get(0); - assertTrue(message.data() instanceof FetchSnapshotResponseData); + assertInstanceOf(FetchSnapshotResponseData.class, message.data()); FetchSnapshotResponseData response = (FetchSnapshotResponseData) message.data(); assertEquals(responseError, Errors.forCode(response.errorCode())); @@ -747,7 +749,7 @@ Optional assertSentFetchSnapshotRes assertEquals(1, sentMessages.size()); RaftMessage message = sentMessages.get(0); - assertTrue(message.data() instanceof FetchSnapshotResponseData); + assertInstanceOf(FetchSnapshotResponseData.class, message.data()); FetchSnapshotResponseData response = (FetchSnapshotResponseData) message.data(); assertEquals(Errors.NONE, Errors.forCode(response.errorCode())); @@ -802,7 +804,7 @@ void discoverLeaderAsObserver( private List collectBeginEpochRequests(int epoch) { List requests = new ArrayList<>(); for (RaftRequest.Outbound raftRequest : channel.drainSentRequests(Optional.of(ApiKeys.BEGIN_QUORUM_EPOCH))) { - assertTrue(raftRequest.data() instanceof BeginQuorumEpochRequestData); + assertInstanceOf(BeginQuorumEpochRequestData.class, raftRequest.data()); BeginQuorumEpochRequestData request = (BeginQuorumEpochRequestData) raftRequest.data(); BeginQuorumEpochRequestData.PartitionData partitionRequest = @@ -967,10 +969,10 @@ void assertFetchRequestData( long fetchOffset, int lastFetchedEpoch ) { - assertTrue( - message.data() instanceof FetchRequestData, - "unexpected request type " + message.data() - ); + assertInstanceOf( + FetchRequestData.class, + message.data(), + "unexpected request type " + message.data()); FetchRequestData request = (FetchRequestData) message.data(); assertEquals(KafkaRaftClient.MAX_FETCH_SIZE_BYTES, request.maxBytes()); assertEquals(fetchMaxWaitMs, request.maxWaitMs()); diff --git a/server-common/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java b/server-common/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java index 36284ed7f3..972ceb1e93 100644 --- a/server-common/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java +++ b/server-common/src/main/java/org/apache/kafka/queue/KafkaEventQueue.java @@ -20,6 +20,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.OptionalLong; import java.util.TreeMap; import java.util.concurrent.RejectedExecutionException; @@ -513,4 +514,36 @@ public void close() throws InterruptedException { eventHandlerThread.join(); log.info("closed event queue."); } + + /** + * Returns the deferred event that the queue is waiting for, idling until + * its deadline comes, if there is any. + * If the queue has immediate work to do, this returns empty. + * This is useful for unit tests, where to make progress, we need to + * speed the clock up until the next scheduled event is ready to run. + */ + public Optional firstDeferredIfIdling() { + lock.lock(); + try { + if (eventHandler.head.next != eventHandler.head) { + // There are events ready to run immediately. The queue is not idling. + return Optional.empty(); + } + Map.Entry entry = eventHandler.deadlineMap.firstEntry(); + if (entry == null) { + // The queue is idling, but not waiting for any deadline. + return Optional.empty(); + } + EventContext eventContext = entry.getValue(); + if (eventContext.insertionType != EventInsertionType.DEFERRED) { + // Any event with a deadline is put in `deadlineMap`. + // But events of type other than DEFERRED will run immediately, + // so the queue will not idle waiting for their deadline. + return Optional.empty(); + } + return Optional.of(eventContext.event); + } finally { + lock.unlock(); + } + } } diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java index 320de2db6b..02cdc1cc87 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java @@ -30,7 +30,7 @@ public final class ServerTopicConfigSynonyms { private static final String LOG_PREFIX = "log."; - private static final String LOG_CLEANER_PREFIX = LOG_PREFIX + "cleaner."; + public static final String LOG_CLEANER_PREFIX = LOG_PREFIX + "cleaner."; /** * Maps topic configurations to their equivalent broker configurations. diff --git a/server-common/src/main/java/org/apache/kafka/server/mutable/BoundedList.java b/server-common/src/main/java/org/apache/kafka/server/mutable/BoundedList.java index 6c5d1ba0d5..d31de006b7 100644 --- a/server-common/src/main/java/org/apache/kafka/server/mutable/BoundedList.java +++ b/server-common/src/main/java/org/apache/kafka/server/mutable/BoundedList.java @@ -40,18 +40,22 @@ public static BoundedList newArrayBacked(int maxLength) { } public static BoundedList newArrayBacked(int maxLength, int initialCapacity) { + if (initialCapacity <= 0) { + throw new IllegalArgumentException("Invalid non-positive initialCapacity of " + initialCapacity); + } return new BoundedList<>(maxLength, new ArrayList<>(initialCapacity)); } - public BoundedList(int maxLength, List underlying) { + private BoundedList(int maxLength, List underlying) { if (maxLength <= 0) { throw new IllegalArgumentException("Invalid non-positive maxLength of " + maxLength); } - this.maxLength = maxLength; + if (underlying.size() > maxLength) { throw new BoundedListTooLongException("Cannot wrap list, because it is longer than " + "the maximum length " + maxLength); } + this.maxLength = maxLength; this.underlying = underlying; } diff --git a/server-common/src/main/java/org/apache/kafka/server/util/FutureUtils.java b/server-common/src/main/java/org/apache/kafka/server/util/FutureUtils.java index a3f78a84b1..a95716407b 100644 --- a/server-common/src/main/java/org/apache/kafka/server/util/FutureUtils.java +++ b/server-common/src/main/java/org/apache/kafka/server/util/FutureUtils.java @@ -19,11 +19,13 @@ import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; +import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import java.util.function.BiConsumer; +import java.util.function.Function; import java.util.function.Supplier; @@ -125,4 +127,21 @@ public static CompletableFuture combineFutures( return res; }); } + + /** + * Applies the given exception handler to all the futures provided in the list + * and returns a new list of futures. + * + * @param futures A list of futures. + * @param fn A function taking an exception to handle it. + * @return A list of futures. + */ + public static List> mapExceptionally( + List> futures, + Function fn + ) { + final List> results = new ArrayList<>(futures.size()); + futures.forEach(future -> results.add(future.exceptionally(fn))); + return results; + } } diff --git a/server-common/src/main/java/org/apache/kafka/server/util/ShutdownableThread.java b/server-common/src/main/java/org/apache/kafka/server/util/ShutdownableThread.java index 06c751e0bb..4268af92f8 100644 --- a/server-common/src/main/java/org/apache/kafka/server/util/ShutdownableThread.java +++ b/server-common/src/main/java/org/apache/kafka/server/util/ShutdownableThread.java @@ -68,6 +68,10 @@ public boolean isShutdownComplete() { return shutdownComplete.getCount() == 0; } + public boolean isStarted() { + return isStarted; + } + /** * @return true if there has been an unexpected error and the thread shut down */ diff --git a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java index add6a88b49..bd9f2594b0 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java @@ -132,6 +132,8 @@ public void testFromVersionString() { assertEquals(IBP_2_6_IV0, MetadataVersion.fromVersionString("2.6")); assertEquals(IBP_2_6_IV0, MetadataVersion.fromVersionString("2.6-IV0")); + // 2.7-IV2 is the latest production version in the 2.7 line + assertEquals(IBP_2_7_IV2, MetadataVersion.fromVersionString("2.7")); assertEquals(IBP_2_7_IV0, MetadataVersion.fromVersionString("2.7-IV0")); assertEquals(IBP_2_7_IV1, MetadataVersion.fromVersionString("2.7-IV1")); assertEquals(IBP_2_7_IV2, MetadataVersion.fromVersionString("2.7-IV2")); @@ -150,24 +152,31 @@ public void testFromVersionString() { assertEquals(IBP_3_2_IV0, MetadataVersion.fromVersionString("3.2")); assertEquals(IBP_3_2_IV0, MetadataVersion.fromVersionString("3.2-IV0")); + // 3.3-IV3 is the latest production version in the 3.3 line + assertEquals(IBP_3_3_IV3, MetadataVersion.fromVersionString("3.3")); assertEquals(IBP_3_3_IV0, MetadataVersion.fromVersionString("3.3-IV0")); assertEquals(IBP_3_3_IV1, MetadataVersion.fromVersionString("3.3-IV1")); assertEquals(IBP_3_3_IV2, MetadataVersion.fromVersionString("3.3-IV2")); assertEquals(IBP_3_3_IV3, MetadataVersion.fromVersionString("3.3-IV3")); + // 3.4-IV0 is the latest production version in the 3.4 line + assertEquals(IBP_3_4_IV0, MetadataVersion.fromVersionString("3.4")); assertEquals(IBP_3_4_IV0, MetadataVersion.fromVersionString("3.4-IV0")); + // 3.5-IV2 is the latest production version in the 3.5 line + assertEquals(IBP_3_5_IV2, MetadataVersion.fromVersionString("3.5")); assertEquals(IBP_3_5_IV0, MetadataVersion.fromVersionString("3.5-IV0")); assertEquals(IBP_3_5_IV1, MetadataVersion.fromVersionString("3.5-IV1")); assertEquals(IBP_3_5_IV2, MetadataVersion.fromVersionString("3.5-IV2")); + // 3.6-IV2 is the latest production version in the 3.6 line + assertEquals(IBP_3_6_IV2, MetadataVersion.fromVersionString("3.6")); assertEquals(IBP_3_6_IV0, MetadataVersion.fromVersionString("3.6-IV0")); assertEquals(IBP_3_6_IV1, MetadataVersion.fromVersionString("3.6-IV1")); assertEquals(IBP_3_6_IV2, MetadataVersion.fromVersionString("3.6-IV2")); // 3.7-IV4 is the latest production version in the 3.7 line assertEquals(IBP_3_7_IV4, MetadataVersion.fromVersionString("3.7")); - assertEquals(IBP_3_7_IV0, MetadataVersion.fromVersionString("3.7-IV0")); assertEquals(IBP_3_7_IV1, MetadataVersion.fromVersionString("3.7-IV1")); assertEquals(IBP_3_7_IV2, MetadataVersion.fromVersionString("3.7-IV2")); diff --git a/server-common/src/test/java/org/apache/kafka/server/mutable/BoundedListTest.java b/server-common/src/test/java/org/apache/kafka/server/mutable/BoundedListTest.java index a5a71f264b..df2608430f 100644 --- a/server-common/src/test/java/org/apache/kafka/server/mutable/BoundedListTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/mutable/BoundedListTest.java @@ -20,9 +20,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; import static org.junit.jupiter.api.Assertions.assertArrayEquals; @@ -33,33 +31,46 @@ @Timeout(120) public class BoundedListTest { + @Test public void testMaxLengthMustNotBeZero() { assertEquals("Invalid non-positive maxLength of 0", assertThrows(IllegalArgumentException.class, - () -> new BoundedList<>(0, new ArrayList())). - getMessage()); + () -> BoundedList.newArrayBacked(0)).getMessage()); + + assertEquals("Invalid non-positive maxLength of 0", + assertThrows(IllegalArgumentException.class, + () -> BoundedList.newArrayBacked(0, 100)).getMessage()); } @Test public void testMaxLengthMustNotBeNegative() { assertEquals("Invalid non-positive maxLength of -123", assertThrows(IllegalArgumentException.class, - () -> new BoundedList<>(-123, new ArrayList())). - getMessage()); + () -> BoundedList.newArrayBacked(-123)).getMessage()); + + assertEquals("Invalid non-positive maxLength of -123", + assertThrows(IllegalArgumentException.class, + () -> BoundedList.newArrayBacked(-123, 100)).getMessage()); + } + + @Test + public void testInitialCapacityMustNotBeZero() { + assertEquals("Invalid non-positive initialCapacity of 0", + assertThrows(IllegalArgumentException.class, + () -> BoundedList.newArrayBacked(100, 0)).getMessage()); } @Test - public void testOwnedListMustNotBeTooLong() { - assertEquals("Cannot wrap list, because it is longer than the maximum length 1", - assertThrows(BoundedListTooLongException.class, - () -> new BoundedList<>(1, new ArrayList<>(Arrays.asList(1, 2)))). - getMessage()); + public void testInitialCapacityMustNotBeNegative() { + assertEquals("Invalid non-positive initialCapacity of -123", + assertThrows(IllegalArgumentException.class, + () -> BoundedList.newArrayBacked(100, -123)).getMessage()); } @Test public void testAddingToBoundedList() { - BoundedList list = new BoundedList<>(2, new ArrayList<>(3)); + BoundedList list = BoundedList.newArrayBacked(2); assertEquals(0, list.size()); assertTrue(list.isEmpty()); assertTrue(list.add(456)); @@ -70,42 +81,42 @@ public void testAddingToBoundedList() { assertEquals("Cannot add another element to the list because it would exceed the " + "maximum length of 2", assertThrows(BoundedListTooLongException.class, - () -> list.add(912)). - getMessage()); + () -> list.add(912)).getMessage()); assertEquals("Cannot add another element to the list because it would exceed the " + "maximum length of 2", assertThrows(BoundedListTooLongException.class, - () -> list.add(0, 912)). - getMessage()); - } - - private static void testHashCodeAndEquals(List a) { - assertEquals(a, new BoundedList<>(123, a)); - assertEquals(a.hashCode(), new BoundedList<>(123, a).hashCode()); - } - - @Test - public void testHashCodeAndEqualsForEmptyList() { - testHashCodeAndEquals(Collections.emptyList()); + () -> list.add(0, 912)).getMessage()); } @Test public void testHashCodeAndEqualsForNonEmptyList() { - testHashCodeAndEquals(Arrays.asList(1, 2, 3, 4, 5, 6, 7)); + BoundedList boundedList = BoundedList.newArrayBacked(7); + List otherList = Arrays.asList(1, 2, 3, 4, 5, 6, 7); + boundedList.addAll(otherList); + + assertEquals(otherList, boundedList); + assertEquals(otherList.hashCode(), boundedList.hashCode()); } @Test public void testSet() { - ArrayList underlying = new ArrayList<>(Arrays.asList(1, 2, 3)); - BoundedList list = new BoundedList<>(3, underlying); - list.set(1, 200); + BoundedList list = BoundedList.newArrayBacked(3); + list.add(1); + list.add(200); + list.add(3); assertEquals(Arrays.asList(1, 200, 3), list); + list.set(0, 100); + list.set(1, 200); + list.set(2, 300); + assertEquals(Arrays.asList(100, 200, 300), list); } @Test public void testRemove() { - ArrayList underlying = new ArrayList<>(Arrays.asList("a", "a", "c")); - BoundedList list = new BoundedList<>(3, underlying); + BoundedList list = BoundedList.newArrayBacked(3); + list.add("a"); + list.add("a"); + list.add("c"); assertEquals(0, list.indexOf("a")); assertEquals(1, list.lastIndexOf("a")); list.remove("a"); @@ -116,8 +127,10 @@ public void testRemove() { @Test public void testClear() { - ArrayList underlying = new ArrayList<>(Arrays.asList("a", "b", "c")); - BoundedList list = new BoundedList<>(3, underlying); + BoundedList list = BoundedList.newArrayBacked(3); + list.add("a"); + list.add("a"); + list.add("c"); list.clear(); assertEquals(Arrays.asList(), list); assertTrue(list.isEmpty()); @@ -125,38 +138,49 @@ public void testClear() { @Test public void testGet() { - BoundedList list = new BoundedList<>(3, Arrays.asList(1, 2, 3)); + BoundedList list = BoundedList.newArrayBacked(3); + list.add(1); + list.add(2); + list.add(3); + assertEquals(1, list.get(0)); assertEquals(2, list.get(1)); + assertEquals(3, list.get(2)); } @Test public void testToArray() { - BoundedList list = new BoundedList<>(3, Arrays.asList(1, 2, 3)); + BoundedList list = BoundedList.newArrayBacked(3); + list.add(1); + list.add(2); + list.add(3); assertArrayEquals(new Integer[] {1, 2, 3}, list.toArray()); assertArrayEquals(new Integer[] {1, 2, 3}, list.toArray(new Integer[3])); } @Test public void testAddAll() { - ArrayList underlying = new ArrayList<>(Arrays.asList("a", "b", "c")); - BoundedList list = new BoundedList<>(5, underlying); + BoundedList list = BoundedList.newArrayBacked(5); + list.add("a"); + list.add("b"); + list.add("c"); assertEquals("Cannot add another 3 element(s) to the list because it would exceed the " + "maximum length of 5", assertThrows(BoundedListTooLongException.class, - () -> list.addAll(Arrays.asList("d", "e", "f"))). - getMessage()); + () -> list.addAll(Arrays.asList("d", "e", "f"))).getMessage()); assertEquals("Cannot add another 3 element(s) to the list because it would exceed the " + "maximum length of 5", assertThrows(BoundedListTooLongException.class, - () -> list.addAll(0, Arrays.asList("d", "e", "f"))). - getMessage()); + () -> list.addAll(0, Arrays.asList("d", "e", "f"))).getMessage()); list.addAll(Arrays.asList("d", "e")); assertEquals(Arrays.asList("a", "b", "c", "d", "e"), list); } @Test public void testIterator() { - BoundedList list = new BoundedList<>(3, Arrays.asList(1, 2, 3)); + BoundedList list = BoundedList.newArrayBacked(3); + list.add(1); + list.add(2); + list.add(3); assertEquals(1, list.iterator().next()); assertEquals(1, list.listIterator().next()); assertEquals(3, list.listIterator(2).next()); @@ -165,7 +189,10 @@ public void testIterator() { @Test public void testIteratorIsImmutable() { - BoundedList list = new BoundedList<>(3, new ArrayList<>(Arrays.asList(1, 2, 3))); + BoundedList list = BoundedList.newArrayBacked(3); + list.add(1); + list.add(2); + list.add(3); assertThrows(UnsupportedOperationException.class, () -> list.iterator().remove()); assertThrows(UnsupportedOperationException.class, @@ -174,7 +201,10 @@ public void testIteratorIsImmutable() { @Test public void testSubList() { - BoundedList list = new BoundedList<>(3, new ArrayList<>(Arrays.asList(1, 2, 3))); + BoundedList list = BoundedList.newArrayBacked(3); + list.add(1); + list.add(2); + list.add(3); assertEquals(Arrays.asList(2), list.subList(1, 2)); assertThrows(UnsupportedOperationException.class, () -> list.subList(1, 2).remove(2)); diff --git a/server-common/src/test/java/org/apache/kafka/server/util/InterBrokerSendThreadTest.java b/server-common/src/test/java/org/apache/kafka/server/util/InterBrokerSendThreadTest.java index f810cc6e63..90b55ed4a1 100644 --- a/server-common/src/test/java/org/apache/kafka/server/util/InterBrokerSendThreadTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/util/InterBrokerSendThreadTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.server.util; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -137,7 +138,7 @@ public void testDisconnectWithoutShutdownShouldCauseException() { Throwable thrown = throwable.get(); assertNotNull(thrown); - assertTrue(thrown instanceof FatalExitError); + assertInstanceOf(FatalExitError.class, thrown); } @Test @@ -317,9 +318,9 @@ public void testInterruption(boolean isShuttingDown) throws InterruptedException final InterBrokerSendThread thread = new TestInterBrokerSendThread(networkClient, t -> { if (isShuttingDown) - assertTrue(t instanceof InterruptedException); + assertInstanceOf(InterruptedException.class, t); else - assertTrue(t instanceof FatalExitError); + assertInstanceOf(FatalExitError.class, t); exception.getAndSet(t); }); diff --git a/server/src/main/java/org/apache/kafka/server/config/Defaults.java b/server/src/main/java/org/apache/kafka/server/config/Defaults.java index b49788933d..84a0e78d1b 100644 --- a/server/src/main/java/org/apache/kafka/server/config/Defaults.java +++ b/server/src/main/java/org/apache/kafka/server/config/Defaults.java @@ -106,13 +106,6 @@ public class Defaults { public static final int NUM_PARTITIONS = 1; public static final String LOG_DIR = "/tmp/kafka-logs"; public static final long LOG_CLEANUP_INTERVAL_MS = 5 * 60 * 1000L; - public static final int LOG_CLEANER_THREADS = 1; - public static final double LOG_CLEANER_IO_MAX_BYTES_PER_SECOND = Double.MAX_VALUE; - public static final long LOG_CLEANER_DEDUPE_BUFFER_SIZE = 128 * 1024 * 1024L; - public static final int LOG_CLEANER_IO_BUFFER_SIZE = 512 * 1024; - public static final double LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR = 0.9d; - public static final int LOG_CLEANER_BACKOFF_MS = 15 * 1000; - public static final boolean LOG_CLEANER_ENABLE = true; public static final int LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS = 60000; public static final int LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS = 60000; public static final int NUM_RECOVERY_THREADS_PER_DATA_DIR = 1; diff --git a/settings.gradle b/settings.gradle index ff5a0a688e..bc3b84c5d9 100644 --- a/settings.gradle +++ b/settings.gradle @@ -96,6 +96,7 @@ include 'clients', 'streams:upgrade-system-tests-34', 'streams:upgrade-system-tests-35', 'streams:upgrade-system-tests-36', + 'streams:upgrade-system-tests-37', 'tools', 'tools:tools-api', 'transaction-coordinator', diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadata.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadata.java index 74d5c3d28a..4d8eaa95bc 100644 --- a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadata.java +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadata.java @@ -32,7 +32,7 @@ public abstract class RemoteLogMetadata { private final int brokerId; /** - * Epoch time in milli seconds at which this event is generated. + * Epoch time in milliseconds at which this event is generated. */ private final long eventTimestampMs; @@ -42,7 +42,7 @@ protected RemoteLogMetadata(int brokerId, long eventTimestampMs) { } /** - * @return Epoch time in milli seconds at which this event is occurred. + * @return Epoch time in milliseconds at which this event is occurred. */ public long eventTimestampMs() { return eventTimestampMs; diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerConfig.java index 9e38d0c02e..8168197fe0 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerConfig.java @@ -16,11 +16,53 @@ */ package org.apache.kafka.storage.internals.log; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.server.config.ServerTopicConfigSynonyms; + /** * Configuration parameters for the log cleaner. */ public class CleanerConfig { public static final String HASH_ALGORITHM = "MD5"; + public static final int LOG_CLEANER_THREADS = 1; + public static final double LOG_CLEANER_IO_MAX_BYTES_PER_SECOND = Double.MAX_VALUE; + public static final long LOG_CLEANER_DEDUPE_BUFFER_SIZE = 128 * 1024 * 1024L; + public static final int LOG_CLEANER_IO_BUFFER_SIZE = 512 * 1024; + public static final double LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR = 0.9d; + public static final int LOG_CLEANER_BACKOFF_MS = 15 * 1000; + public static final boolean LOG_CLEANER_ENABLE = true; + + public static final String LOG_CLEANER_THREADS_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "threads"; + public static final String LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "io.max.bytes.per.second"; + public static final String LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "dedupe.buffer.size"; + public static final String LOG_CLEANER_IO_BUFFER_SIZE_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "io.buffer.size"; + public static final String LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "io.buffer.load.factor"; + public static final String LOG_CLEANER_BACKOFF_MS_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "backoff.ms"; + public static final String LOG_CLEANER_MIN_CLEAN_RATIO_PROP = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG); + public static final String LOG_CLEANER_ENABLE_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "enable"; + public static final String LOG_CLEANER_DELETE_RETENTION_MS_PROP = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.DELETE_RETENTION_MS_CONFIG); + public static final String LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG); + public static final String LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG); + + public static final String LOG_CLEANER_MIN_CLEAN_RATIO_DOC = "The minimum ratio of dirty log to total log for a log to eligible for cleaning. " + + "If the " + LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP + " or the " + LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP + + " configurations are also specified, then the log compactor considers the log eligible for compaction " + + "as soon as either: (i) the dirty ratio threshold has been met and the log has had dirty (uncompacted) " + + "records for at least the " + LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP + " duration, or (ii) if the log has had " + + "dirty (uncompacted) records for at most the " + LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP + " period."; + public static final String LOG_CLEANER_THREADS_DOC = "The number of background threads to use for log cleaning"; + public static final String LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_DOC = "The log cleaner will be throttled so that the sum of its read and write i/o will be less than this value on average"; + public static final String LOG_CLEANER_DEDUPE_BUFFER_SIZE_DOC = "The total memory used for log deduplication across all cleaner threads"; + public static final String LOG_CLEANER_IO_BUFFER_SIZE_DOC = "The total memory used for log cleaner I/O buffers across all cleaner threads"; + public static final String LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_DOC = "Log cleaner dedupe buffer load factor. The percentage full the dedupe buffer can become. A higher value " + + "will allow more log to be cleaned at once but will lead to more hash collisions"; + public static final String LOG_CLEANER_BACKOFF_MS_DOC = "The amount of time to sleep when there are no logs to clean"; + public static final String LOG_CLEANER_ENABLE_DOC = "Enable the log cleaner process to run on the server. Should be enabled if using any topics with a cleanup.policy=compact including the internal offsets topic. If disabled those topics will not be compacted and continually grow in size."; + public static final String LOG_CLEANER_DELETE_RETENTION_MS_DOC = "The amount of time to retain tombstone message markers for log compacted topics. This setting also gives a bound " + + "on the time in which a consumer must complete a read if they begin from offset 0 to ensure that they get a valid snapshot of the final stage (otherwise " + + "tombstones messages may be collected before a consumer completes their scan)."; + public static final String LOG_CLEANER_MIN_COMPACTION_LAG_MS_DOC = "The minimum time a message will remain uncompacted in the log. Only applicable for logs that are being compacted."; + public static final String LOG_CLEANER_MAX_COMPACTION_LAG_MS_DOC = "The maximum time a message will remain ineligible for compaction in the log. Only applicable for logs that are being compacted."; public final int numThreads; public final long dedupeBufferSize; diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java index 685a088100..928f7149a1 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java @@ -129,12 +129,6 @@ public LogSegment(FileRecords log, this.created = time.milliseconds(); } - // Visible for testing - public LogSegment(LogSegment segment) { - this(segment.log, segment.lazyOffsetIndex, segment.lazyTimeIndex, segment.txnIndex, segment.baseOffset, - segment.indexIntervalBytes, segment.rollJitterMs, segment.time); - } - public OffsetIndex offsetIndex() throws IOException { return lazyOffsetIndex.get(); } @@ -239,17 +233,17 @@ private boolean canConvertToRelativeOffset(long offset) throws IOException { * * @param largestOffset The last offset in the message set * @param largestTimestampMs The largest timestamp in the message set. - * @param shallowOffsetOfMaxTimestamp The offset of the message that has the largest timestamp in the messages to append. + * @param offsetOfMaxTimestamp The offset of the message that has the largest timestamp in the messages to append. * @param records The log entries to append. * @throws LogSegmentOffsetOverflowException if the largest offset causes index offset overflow */ public void append(long largestOffset, long largestTimestampMs, - long shallowOffsetOfMaxTimestamp, + long offsetOfMaxTimestamp, MemoryRecords records) throws IOException { if (records.sizeInBytes() > 0) { - LOGGER.trace("Inserting {} bytes at end offset {} at position {} with largest timestamp {} at shallow offset {}", - records.sizeInBytes(), largestOffset, log.sizeInBytes(), largestTimestampMs, shallowOffsetOfMaxTimestamp); + LOGGER.trace("Inserting {} bytes at end offset {} at position {} with largest timestamp {} at offset {}", + records.sizeInBytes(), largestOffset, log.sizeInBytes(), largestTimestampMs, offsetOfMaxTimestamp); int physicalPosition = log.sizeInBytes(); if (physicalPosition == 0) rollingBasedTimestamp = OptionalLong.of(largestTimestampMs); @@ -261,7 +255,7 @@ public void append(long largestOffset, LOGGER.trace("Appended {} to {} at end offset {}", appendedBytes, log.file(), largestOffset); // Update the in memory max timestamp and corresponding offset. if (largestTimestampMs > maxTimestampSoFar()) { - maxTimestampAndOffsetSoFar = new TimestampOffset(largestTimestampMs, shallowOffsetOfMaxTimestamp); + maxTimestampAndOffsetSoFar = new TimestampOffset(largestTimestampMs, offsetOfMaxTimestamp); } // append an entry to the index (if needed) if (bytesSinceLastIndexEntry > indexIntervalBytes) { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java index 378247fa27..9aa1e06633 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java @@ -68,17 +68,17 @@ public static class ValidationResult { public final long logAppendTimeMs; public final MemoryRecords validatedRecords; public final long maxTimestampMs; - public final long shallowOffsetOfMaxTimestampMs; + public final long offsetOfMaxTimestampMs; public final boolean messageSizeMaybeChanged; public final RecordValidationStats recordValidationStats; public ValidationResult(long logAppendTimeMs, MemoryRecords validatedRecords, long maxTimestampMs, - long shallowOffsetOfMaxTimestampMs, boolean messageSizeMaybeChanged, + long offsetOfMaxTimestampMs, boolean messageSizeMaybeChanged, RecordValidationStats recordValidationStats) { this.logAppendTimeMs = logAppendTimeMs; this.validatedRecords = validatedRecords; this.maxTimestampMs = maxTimestampMs; - this.shallowOffsetOfMaxTimestampMs = shallowOffsetOfMaxTimestampMs; + this.offsetOfMaxTimestampMs = offsetOfMaxTimestampMs; this.messageSizeMaybeChanged = messageSizeMaybeChanged; this.recordValidationStats = recordValidationStats; } @@ -149,7 +149,7 @@ public LogValidator(MemoryRecords records, * avoid expensive re-compression. * * Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset - * of the shallow message with the max timestamp and a boolean indicating whether the message sizes may have changed. + * of the message with the max timestamp and a boolean indicating whether the message sizes may have changed. */ public ValidationResult validateMessagesAndAssignOffsets(PrimitiveRef.LongRef offsetCounter, MetricsRecorder metricsRecorder, @@ -232,7 +232,7 @@ private ValidationResult convertAndAssignOffsetsNonCompressed(LongRef offsetCoun now, convertedRecords, info.maxTimestamp, - info.shallowOffsetOfMaxTimestamp, + info.offsetOfMaxTimestamp, true, recordValidationStats); } @@ -296,10 +296,6 @@ public ValidationResult assignOffsetsNonCompressed(LongRef offsetCounter, offsetOfMaxTimestamp = initialOffset; } - if (toMagic >= RecordBatch.MAGIC_VALUE_V2) { - offsetOfMaxTimestamp = offsetCounter.value - 1; - } - return new ValidationResult( now, records, @@ -330,6 +326,8 @@ public ValidationResult validateMessagesAndAssignOffsetsCompressed(LongRef offse long maxTimestamp = RecordBatch.NO_TIMESTAMP; LongRef expectedInnerOffset = PrimitiveRef.ofLong(0); List validatedRecords = new ArrayList<>(); + long offsetOfMaxTimestamp = -1; + long initialOffset = offsetCounter.value; int uncompressedSizeInBytes = 0; @@ -379,8 +377,11 @@ public ValidationResult validateMessagesAndAssignOffsetsCompressed(LongRef offse && batch.magic() > RecordBatch.MAGIC_VALUE_V0 && toMagic > RecordBatch.MAGIC_VALUE_V0) { - if (record.timestamp() > maxTimestamp) + if (record.timestamp() > maxTimestamp) { maxTimestamp = record.timestamp(); + // The offset is only increased when it is a valid record + offsetOfMaxTimestamp = initialOffset + validatedRecords.size(); + } // Some older clients do not implement the V1 internal offsets correctly. // Historically the broker handled this by rewriting the batches rather @@ -417,8 +418,10 @@ public ValidationResult validateMessagesAndAssignOffsetsCompressed(LongRef offse long lastOffset = offsetCounter.value - 1; firstBatch.setLastOffset(lastOffset); - if (timestampType == TimestampType.LOG_APPEND_TIME) + if (timestampType == TimestampType.LOG_APPEND_TIME) { maxTimestamp = now; + offsetOfMaxTimestamp = initialOffset; + } if (toMagic >= RecordBatch.MAGIC_VALUE_V1) firstBatch.setMaxTimestamp(timestampType, maxTimestamp); @@ -431,7 +434,7 @@ public ValidationResult validateMessagesAndAssignOffsetsCompressed(LongRef offse now, records, maxTimestamp, - lastOffset, + offsetOfMaxTimestamp, false, recordValidationStats); } @@ -473,7 +476,7 @@ private ValidationResult buildRecordsAndAssignOffsets(LongRef offsetCounter, logAppendTime, records, info.maxTimestamp, - info.shallowOffsetOfMaxTimestamp, + info.offsetOfMaxTimestamp, true, recordValidationStats); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java index abd4957213..3ee101ff6a 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java @@ -614,7 +614,7 @@ public void deleteSnapshotsBefore(long offset) throws IOException { } public Optional fetchSnapshot(long offset) { - return Optional.of(snapshots.get(offset)).map(x -> x.file()); + return Optional.ofNullable(snapshots.get(offset)).map(x -> x.file()); } private Optional oldestSnapshotFile() { 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 2f4841269b..44c064f40f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -418,6 +418,12 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String UPGRADE_FROM_36 = UpgradeFromValues.UPGRADE_FROM_36.toString(); + /** + * Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 3.7.x}. + */ + @SuppressWarnings("WeakerAccess") + public static final String UPGRADE_FROM_37 = UpgradeFromValues.UPGRADE_FROM_37.toString(); + /** * Config value for parameter {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} for at-least-once processing guarantees. */ @@ -507,13 +513,14 @@ public class StreamsConfig extends AbstractConfig { /** {@code client.id} */ @SuppressWarnings("WeakerAccess") public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; - private static final String CLIENT_ID_DOC = "An ID prefix string used for the client IDs of internal [main-|restore-|global-]consumer, producer, and admin clients" + - " with pattern <client.id>-[Global]StreamThread[-<threadSequenceNumber$gt;]-<consumer|producer|restore-consumer|global-consumer>."; + private static final String CLIENT_ID_DOC = "An ID prefix string used for the client IDs of internal (main, restore, and global) consumers , producers, and admin clients" + + " with pattern <client.id>-[Global]StreamThread[-<threadSequenceNumber>]-<consumer|producer|restore-consumer|global-consumer>."; /** {@code enable.metrics.push} */ @SuppressWarnings("WeakerAccess") public static final String ENABLE_METRICS_PUSH_CONFIG = CommonClientConfigs.ENABLE_METRICS_PUSH_CONFIG; - public static final String ENABLE_METRICS_PUSH_DOC = "Whether to enable pushing of internal [main-|restore-|global]consumer, producer, and admin client metrics to the cluster, if the cluster has a client metrics subscription which matches a client."; + public static final String ENABLE_METRICS_PUSH_DOC = "Whether to enable pushing of internal client metrics for (main, restore, and global) consumers, producers, and admin clients." + + " The cluster must have a client metrics subscription which corresponds to a client."; /** {@code commit.interval.ms} */ @SuppressWarnings("WeakerAccess") @@ -774,7 +781,7 @@ public class StreamsConfig extends AbstractConfig { UPGRADE_FROM_25 + "\", \"" + UPGRADE_FROM_26 + "\", \"" + UPGRADE_FROM_27 + "\", \"" + UPGRADE_FROM_28 + "\", \"" + UPGRADE_FROM_30 + "\", \"" + UPGRADE_FROM_31 + "\", \"" + UPGRADE_FROM_32 + "\", \"" + UPGRADE_FROM_33 + "\", \"" + UPGRADE_FROM_34 + "\", \"" + - UPGRADE_FROM_35 + "\", \"" + UPGRADE_FROM_36 + "(for upgrading from the corresponding old version)."; + UPGRADE_FROM_35 + "\", \"" + UPGRADE_FROM_36 + "\", \"" + UPGRADE_FROM_37 + "(for upgrading from the corresponding old version)."; /** {@code windowstore.changelog.additional.retention.ms} */ @SuppressWarnings("WeakerAccess") @@ -878,7 +885,8 @@ public class StreamsConfig extends AbstractConfig { Type.STRING, "", Importance.MEDIUM, - CLIENT_ID_DOC) + CLIENT_ID_DOC, + "<application.id>-<random-UUID>") .define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, Type.CLASS, LogAndFailExceptionHandler.class.getName(), 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 d9c810aa42..d1f0d1eb8b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/Topology.java +++ b/streams/src/main/java/org/apache/kafka/streams/Topology.java @@ -738,6 +738,88 @@ public synchronized Topology addStateStore(final StoreBuilder storeBuilder, return this; } + /** + * Adds a read-only {@link StateStore} to the topology. + *

    + * A read-only {@link StateStore} does not create a dedicated changelog topic but uses it's input topic as + * changelog; thus, the used topic should be configured with log compaction. + *

    + * The auto.offset.reset property will be set to earliest for this topic. + *

    + * The provided {@link ProcessorSupplier} will be used to create a processor for all messages received + * from the given topic. This processor should contain logic to keep the {@link StateStore} up-to-date. + * + * @param storeBuilder user defined store builder + * @param sourceName name of the {@link SourceNode} that will be automatically added + * @param timestampExtractor the stateless timestamp extractor used for this source, + * if not specified the default extractor defined in the configs will be used + * @param keyDeserializer the {@link Deserializer} to deserialize keys with + * @param valueDeserializer the {@link Deserializer} to deserialize values with + * @param topic the topic to source the data from + * @param processorName the name of the {@link ProcessorSupplier} + * @param stateUpdateSupplier the instance of {@link ProcessorSupplier} + * @return itself + * @throws TopologyException if the processor of state is already registered + */ + public synchronized Topology addReadOnlyStateStore(final StoreBuilder storeBuilder, + final String sourceName, + final TimestampExtractor timestampExtractor, + final Deserializer keyDeserializer, + final Deserializer valueDeserializer, + final String topic, + final String processorName, + final ProcessorSupplier stateUpdateSupplier) { + storeBuilder.withLoggingDisabled(); + + internalTopologyBuilder.addSource(AutoOffsetReset.EARLIEST, sourceName, timestampExtractor, keyDeserializer, valueDeserializer, topic); + internalTopologyBuilder.addProcessor(processorName, stateUpdateSupplier, sourceName); + internalTopologyBuilder.addStateStore(storeBuilder, processorName); + internalTopologyBuilder.connectSourceStoreAndTopic(storeBuilder.name(), topic); + + return this; + } + + /** + * Adds a read-only {@link StateStore} to the topology. + *

    + * A read-only {@link StateStore} does not create a dedicated changelog topic but uses it's input topic as + * changelog; thus, the used topic should be configured with log compaction. + *

    + * The auto.offset.reset property will be set to earliest for this topic. + *

    + * The provided {@link ProcessorSupplier} will be used to create a processor for all messages received + * from the given topic. This processor should contain logic to keep the {@link StateStore} up-to-date. + * The default {@link TimestampExtractor} as specified in the {@link StreamsConfig config} is used. + * + * @param storeBuilder user defined store builder + * @param sourceName name of the {@link SourceNode} that will be automatically added + * @param keyDeserializer the {@link Deserializer} to deserialize keys with + * @param valueDeserializer the {@link Deserializer} to deserialize values with + * @param topic the topic to source the data from + * @param processorName the name of the {@link ProcessorSupplier} + * @param stateUpdateSupplier the instance of {@link ProcessorSupplier} + * @return itself + * @throws TopologyException if the processor of state is already registered + */ + public synchronized Topology addReadOnlyStateStore(final StoreBuilder storeBuilder, + final String sourceName, + final Deserializer keyDeserializer, + final Deserializer valueDeserializer, + final String topic, + final String processorName, + final ProcessorSupplier stateUpdateSupplier) { + return addReadOnlyStateStore( + storeBuilder, + sourceName, + null, + keyDeserializer, + valueDeserializer, + topic, + processorName, + stateUpdateSupplier + ); + } + /** * Adds a global {@link StateStore} to the topology. * The {@link StateStore} sources its data from all partitions of the provided input topic. diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java b/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java index abc2c7cb45..2bf19da39b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java @@ -38,7 +38,8 @@ public enum UpgradeFromValues { UPGRADE_FROM_33("3.3"), UPGRADE_FROM_34("3.4"), UPGRADE_FROM_35("3.5"), - UPGRADE_FROM_36("3.6"); + UPGRADE_FROM_36("3.6"), + UPGRADE_FROM_37("3.7"); private final String value; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 603e1e8255..124386b9bc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -51,7 +51,8 @@ class KStreamKStreamJoin implements ProcessorSupplier implements ProcessorSupplier record) { return; } - boolean needOuterJoin = outer; // Emit all non-joined records which window has closed if (inputRecordTimestamp == sharedTimeTracker.streamTime) { outerJoinStore.ifPresent(store -> emitNonJoinedOuterRecords(store, record)); } + + boolean needOuterJoin = outer; try (final WindowStoreIterator iter = otherWindowStore.fetch(record.key(), timeFrom, timeTo)) { while (iter.hasNext()) { needOuterJoin = false; @@ -200,7 +202,7 @@ private void emitNonJoinedOuterRecords( // to reduce runtime cost, we try to avoid paying those cost // only try to emit left/outer join results if there _might_ be any result records - if (sharedTimeTracker.minTime >= sharedTimeTracker.streamTime - joinSpuriousLookBackTimeMs - joinGraceMs) { + if (sharedTimeTracker.minTime + joinAfterMs + joinGraceMs >= sharedTimeTracker.streamTime) { return; } // throttle the emit frequency to a (configurable) interval; @@ -222,6 +224,8 @@ private void emitNonJoinedOuterRecords( TimestampedKeyAndJoinSide prevKey = null; while (it.hasNext()) { + boolean outerJoinLeftBreak = false; + boolean outerJoinRightBreak = false; final KeyValue, LeftOrRightValue> next = it.next(); final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide = next.key; final LeftOrRightValue value = next.value; @@ -230,8 +234,19 @@ private void emitNonJoinedOuterRecords( sharedTimeTracker.minTime = timestamp; // Skip next records if window has not closed - if (timestamp + joinAfterMs + joinGraceMs >= sharedTimeTracker.streamTime) { - break; + final long outerJoinLookBackTimeMs = getOuterJoinLookBackTimeMs(timestampedKeyAndJoinSide); + if (sharedTimeTracker.minTime + outerJoinLookBackTimeMs + joinGraceMs >= sharedTimeTracker.streamTime) { + if (timestampedKeyAndJoinSide.isLeftSide()) { + outerJoinLeftBreak = true; // there are no more candidates to emit on left-outerJoin-side + } else { + outerJoinRightBreak = true; // there are no more candidates to emit on right-outerJoin-side + } + if (outerJoinLeftBreak && outerJoinRightBreak) { + break; // there are no more candidates to emit on left-outerJoin-side and + // right-outerJoin-side + } else { + continue; // there are possibly candidates left on the other outerJoin-side + } } final VOut nullJoinedValue; @@ -268,6 +283,15 @@ private void emitNonJoinedOuterRecords( } } + private long getOuterJoinLookBackTimeMs(final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide) { + // depending on the JoinSide we fill in the outerJoinLookBackTimeMs + if (timestampedKeyAndJoinSide.isLeftSide()) { + return windowsAfterMs; // On the left-JoinSide we look back in time + } else { + return windowsBeforeMs; // On the right-JoinSide we look forward in time + } + } + @Override public void close() { sharedTimeTrackerSupplier.remove(context().taskId()); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplier.java index 600b28078b..cbb66f98fa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplier.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream.internals.foreignkeyjoin; import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.internals.KTableValueGetter; @@ -27,6 +28,8 @@ import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.internals.Murmur3; import org.slf4j.Logger; @@ -71,6 +74,8 @@ public Processor, K, VR> get() { private Serializer runtimeValueSerializer = constructionTimeValueSerializer; private KTableValueGetter valueGetter; + private Sensor droppedRecordsSensor; + @SuppressWarnings("unchecked") @Override @@ -82,6 +87,13 @@ public void init(final ProcessorContext context) { if (runtimeValueSerializer == null) { runtimeValueSerializer = (Serializer) context.valueSerde().serializer(); } + + final InternalProcessorContext internalProcessorContext = (InternalProcessorContext) context; + droppedRecordsSensor = TaskMetrics.droppedRecordsSensor( + Thread.currentThread().getName(), + internalProcessorContext.taskId().toString(), + internalProcessorContext.metrics() + ); } @Override @@ -112,6 +124,7 @@ public void process(final Record> record) { context().forward(record.withValue(result)); } else { LOG.trace("Dropping FK-join response due to hash mismatch. Expected {}. Actual {}", messageHash, currentHash); + droppedRecordsSensor.record(); } } }; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinProcessorSupplier.java index a8677ce295..388b669e98 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinProcessorSupplier.java @@ -107,7 +107,11 @@ public void process(final Record, Change(value.getHash(), valueToSend, value.getPrimaryPartition())) + .withValue(new SubscriptionResponseWrapper<>( + value.getHash(), + valueToSend, + value.getPrimaryPartition() + )) .withTimestamp(resultTimestamp) ); break; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateMaintainer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateMaintainer.java index 9a8aab6eb3..06afb6fde4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateMaintainer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateMaintainer.java @@ -34,4 +34,6 @@ interface GlobalStateMaintainer { void close(final boolean wipeStateStore) throws IOException; void update(ConsumerRecord record); + + void maybeCheckpoint(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java index 523228542a..da7ebba209 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; 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.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; @@ -45,18 +46,26 @@ public class GlobalStateUpdateTask implements GlobalStateMaintainer { private final Map deserializers = new HashMap<>(); private final GlobalStateManager stateMgr; private final DeserializationExceptionHandler deserializationExceptionHandler; + private final Time time; + private final long flushInterval; + private long lastFlush; public GlobalStateUpdateTask(final LogContext logContext, final ProcessorTopology topology, final InternalProcessorContext processorContext, final GlobalStateManager stateMgr, - final DeserializationExceptionHandler deserializationExceptionHandler) { + final DeserializationExceptionHandler deserializationExceptionHandler, + final Time time, + final long flushInterval + ) { this.logContext = logContext; this.log = logContext.logger(getClass()); this.topology = topology; this.stateMgr = stateMgr; this.processorContext = processorContext; this.deserializationExceptionHandler = deserializationExceptionHandler; + this.time = time; + this.flushInterval = flushInterval; } /** @@ -86,6 +95,7 @@ public Map initialize() { } initTopology(); processorContext.initialize(); + lastFlush = time.milliseconds(); return stateMgr.changelogOffsets(); } @@ -150,5 +160,13 @@ private void initTopology() { } } + @Override + public void maybeCheckpoint() { + final long now = time.milliseconds(); + if (now - flushInterval >= lastFlush && StateManagerUtil.checkpointNeeded(false, stateMgr.changelogOffsets(), offsets)) { + flushState(); + lastFlush = now; + } + } } 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 82a0cc5113..1ed517b15d 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 @@ -228,25 +228,17 @@ public GlobalStreamThread(final ProcessorTopology topology, static class StateConsumer { private final Consumer globalConsumer; private final GlobalStateMaintainer stateMaintainer; - private final Time time; private final Duration pollTime; - private final long flushInterval; private final Logger log; - private long lastFlush; - StateConsumer(final LogContext logContext, final Consumer globalConsumer, final GlobalStateMaintainer stateMaintainer, - final Time time, - final Duration pollTime, - final long flushInterval) { + final Duration pollTime) { this.log = logContext.logger(getClass()); this.globalConsumer = globalConsumer; this.stateMaintainer = stateMaintainer; - this.time = time; this.pollTime = pollTime; - this.flushInterval = flushInterval; } /** @@ -259,7 +251,6 @@ void initialize() { for (final Map.Entry entry : partitionOffsets.entrySet()) { globalConsumer.seek(entry.getKey(), entry.getValue()); } - lastFlush = time.milliseconds(); } void pollAndUpdate() { @@ -267,11 +258,7 @@ void pollAndUpdate() { for (final ConsumerRecord record : received) { stateMaintainer.update(record); } - final long now = time.milliseconds(); - if (now - flushInterval >= lastFlush) { - stateMaintainer.flushState(); - lastFlush = now; - } + stateMaintainer.maybeCheckpoint(); } public void close(final boolean wipeStateStore) throws IOException { @@ -418,11 +405,11 @@ private StateConsumer initialize() { topology, globalProcessorContext, stateMgr, - config.defaultDeserializationExceptionHandler() + config.defaultDeserializationExceptionHandler(), + time, + config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG) ), - time, - Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG)), - config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG) + Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG)) ); try { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java index b68b24036f..839baaad87 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java @@ -193,12 +193,14 @@ public boolean equals(final Object o) { } /** - * Equality is implemented in support of tests, *not* for use in Hash collections, since this class is mutable. + * Equality is implemented in support of tests, *not* for use in Hash collections, since this class is mutable + * due to the {@link Headers} field it contains. */ @Deprecated @Override public int hashCode() { - throw new UnsupportedOperationException("ProcessorRecordContext is unsafe for use in Hash collections"); + throw new UnsupportedOperationException("ProcessorRecordContext is unsafe for use in Hash collections " + + "due to the mutable Headers field"); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index 3f16838dfc..fb4e45cdfb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -628,7 +628,7 @@ private boolean assignTasksToClients(final Cluster fullMetadata, log.info("{} client nodes and {} consumers participating in this rebalance: \n{}.", clientStates.size(), - clientStates.values().stream().map(ClientState::capacity).reduce(Integer::sum), + clientStates.values().stream().map(ClientState::capacity).reduce(Integer::sum).orElse(0), clientStates.entrySet().stream() .sorted(comparingByKey()) .map(entry -> entry.getKey() + ": " + entry.getValue().consumers()) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java index 039fd2b258..6d99d93536 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java @@ -127,6 +127,7 @@ public RebalanceProtocol rebalanceProtocol() { case UPGRADE_FROM_34: case UPGRADE_FROM_35: case UPGRADE_FROM_36: + case UPGRADE_FROM_37: // we need to add new version when new "upgrade.from" values become available // This config is for explicitly sending FK response to a requested partition @@ -187,6 +188,7 @@ public int configuredMetadataVersion(final int priorVersion) { case UPGRADE_FROM_34: case UPGRADE_FROM_35: case UPGRADE_FROM_36: + case UPGRADE_FROM_37: // we need to add new version when new "upgrade.from" values become available // This config is for explicitly sending FK response to a requested partition diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java index f905a29625..4ef656c212 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java @@ -93,7 +93,7 @@ default KeyValueIterator reverseRange(K from, K to) { * and must not return null values. * Order is not guaranteed as bytes lexicographical ordering might not represent key order. * - * @return An reverse iterator of all key/value pairs in the store, from largest to smallest key bytes. + * @return A reverse iterator of all key/value pairs in the store, from largest to smallest key bytes. * @throws InvalidStateStoreException if the store is not initialized */ default KeyValueIterator reverseAll() { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java index 7fe11a6bea..5a52f00f60 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java @@ -37,7 +37,7 @@ public interface ReadOnlySessionStore { * is the upper bound of the search interval, and the method returns all sessions that overlap * with the search interval. * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime - * if won't be contained in the result: + * it won't be contained in the result: *

    {@code
          * earliestSessionEndTime: ESET
          * latestSessionStartTime: LSST
    @@ -48,7 +48,6 @@ public interface ReadOnlySessionStore {
          * 

    * This iterator must be closed after use. * - * * @param key the key to return sessions for * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where * iteration starts. @@ -72,7 +71,7 @@ default KeyValueIterator, AGG> findSessions(final K key, * is the upper bound of the search interval, and the method returns all sessions that overlap * with the search interval. * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime - * if won't be contained in the result: + * it won't be contained in the result: *

    {@code
          * earliestSessionEndTime: ESET
          * latestSessionStartTime: LSST
    @@ -106,7 +105,7 @@ default KeyValueIterator, AGG> findSessions(final K key,
          * is the upper bound of the search interval, and the method returns all sessions that overlap
          * with the search interval.
          * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime
    -     * if won't be contained in the result:
    +     * it won't be contained in the result:
          * 
    {@code
          * earliestSessionEndTime: ESET
          * latestSessionStartTime: LSST
    @@ -140,7 +139,7 @@ default KeyValueIterator, AGG> backwardFindSessions(final K key,
          * is the upper bound of the search interval, and the method returns all sessions that overlap
          * with the search interval.
          * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime
    -     * if won't be contained in the result:
    +     * it won't be contained in the result:
          * 
    {@code
          * earliestSessionEndTime: ESET
          * latestSessionStartTime: LSST
    @@ -175,7 +174,7 @@ default KeyValueIterator, AGG> backwardFindSessions(final K key,
          * is the upper bound of the search interval, and the method returns all sessions that overlap
          * with the search interval.
          * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime
    -     * if won't be contained in the result:
    +     * it won't be contained in the result:
          * 
    {@code
          * earliestSessionEndTime: ESET
          * latestSessionStartTime: LSST
    @@ -213,7 +212,7 @@ default KeyValueIterator, AGG> findSessions(final K keyFrom,
          * is the upper bound of the search interval, and the method returns all sessions that overlap
          * with the search interval.
          * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime
    -     * if won't be contained in the result:
    +     * it won't be contained in the result:
          * 
    {@code
          * earliestSessionEndTime: ESET
          * latestSessionStartTime: LSST
    @@ -251,7 +250,7 @@ default KeyValueIterator, AGG> findSessions(final K keyFrom,
          * is the upper bound of the search interval, and the method returns all sessions that overlap
          * with the search interval.
          * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime
    -     * if won't be contained in the result:
    +     * it won't be contained in the result:
          * 
    {@code
          * earliestSessionEndTime: ESET
          * latestSessionStartTime: LSST
    @@ -289,7 +288,7 @@ default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom,
          * is the upper bound of the search interval, and the method returns all sessions that overlap
          * with the search interval.
          * Thus, if a session ends before earliestSessionEndTime, or starts after latestSessionStartTime
    -     * if won't be contained in the result:
    +     * it won't be contained in the result:
          * 
    {@code
          * earliestSessionEndTime: ESET
          * latestSessionStartTime: LSST
    diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java
    index 3df170d5ab..30ca72a64c 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java
    @@ -174,7 +174,7 @@ default KeyValueIterator, V> backwardFetch(K keyFrom, K keyTo, Insta
          * Gets all the key-value pairs in the existing windows in backward order
          * with respect to time (from end to beginning of time).
          *
    -     * @return an backward iterator over windowed key-value pairs {@code , value>}, from the end to beginning of time.
    +     * @return a backward iterator over windowed key-value pairs {@code , value>}, from the end to beginning of time.
          * @throws InvalidStateStoreException if the store is not initialized
          */
         default KeyValueIterator, V> backwardAll() {
    @@ -199,7 +199,7 @@ default KeyValueIterator, V> backwardAll() {
          *
          * @param timeFrom the beginning of the time slot from which to search (inclusive), where iteration ends.
          * @param timeTo   the end of the time slot from which to search (inclusive), where iteration starts.
    -     * @return an backward iterator over windowed key-value pairs {@code , value>}, from end to beginning of time.
    +     * @return a backward iterator over windowed key-value pairs {@code , value>}, from end to beginning of time.
          * @throws InvalidStateStoreException if the store is not initialized
          * @throws NullPointerException       if {@code null} is used for any key
          * @throws IllegalArgumentException   if duration is negative or can't be represented as {@code long milliseconds}
    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 3f1bfef631..e7353002b2 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
    @@ -87,6 +87,7 @@
     import static org.junit.Assert.assertThrows;
     import static org.junit.Assert.assertTrue;
     import static org.junit.Assert.fail;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     
     public class StreamsConfigTest {
         @Rule
    @@ -870,16 +871,16 @@ public void shouldUseNewConfigsWhenPresent() {
             props.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class);
     
             final StreamsConfig config = new StreamsConfig(props);
    -        assertTrue(config.defaultKeySerde() instanceof Serdes.LongSerde);
    -        assertTrue(config.defaultValueSerde() instanceof Serdes.LongSerde);
    -        assertTrue(config.defaultTimestampExtractor() instanceof MockTimestampExtractor);
    +        assertInstanceOf(Serdes.LongSerde.class, config.defaultKeySerde());
    +        assertInstanceOf(Serdes.LongSerde.class, config.defaultValueSerde());
    +        assertInstanceOf(MockTimestampExtractor.class, config.defaultTimestampExtractor());
         }
     
         @Test
         public void shouldUseCorrectDefaultsWhenNoneSpecified() {
             final StreamsConfig config = new StreamsConfig(getStreamsConfig());
     
    -        assertTrue(config.defaultTimestampExtractor() instanceof FailOnInvalidTimestamp);
    +        assertInstanceOf(FailOnInvalidTimestamp.class, config.defaultTimestampExtractor());
             assertThrows(ConfigException.class, config::defaultKeySerde);
             assertThrows(ConfigException.class, config::defaultValueSerde);
         }
    @@ -1460,7 +1461,7 @@ public void shouldReturnRackAwareAssignmentNonOverlapCost() {
         @Test
         public void shouldReturnDefaultClientSupplier() {
             final KafkaClientSupplier supplier = streamsConfig.getKafkaClientSupplier();
    -        assertTrue(supplier instanceof DefaultKafkaClientSupplier);
    +        assertInstanceOf(DefaultKafkaClientSupplier.class, supplier);
         }
     
         @Test
    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 5fdf5c220c..ceecbc3bc2 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java
    @@ -38,6 +38,7 @@
     import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
     import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.SubtopologyDescription;
     import org.apache.kafka.streams.processor.internals.ProcessorTopology;
    +import org.apache.kafka.streams.processor.internals.StoreFactory;
     import org.apache.kafka.streams.state.KeyValueStore;
     import org.apache.kafka.streams.state.SessionStore;
     import org.apache.kafka.streams.state.StoreBuilder;
    @@ -58,6 +59,7 @@
     import org.junit.runner.RunWith;
     import org.mockito.Mock;
     import org.mockito.junit.MockitoJUnitRunner;
    +import org.mockito.internal.util.collections.Sets;
     
     import java.time.Duration;
     import java.util.Arrays;
    @@ -337,7 +339,7 @@ public void shouldNotAllowToAddStoreWithSameNameAndDifferentInstance() {
             mockStoreBuilder();
             topology.addStateStore(storeBuilder);
     
    -        final StoreBuilder otherStoreBuilder = mock(StoreBuilder.class);
    +        final StoreBuilder otherStoreBuilder = mock(StoreBuilder.class);
             when(otherStoreBuilder.name()).thenReturn("store");
             when(otherStoreBuilder.logConfig()).thenReturn(Collections.emptyMap());
             when(otherStoreBuilder.loggingEnabled()).thenReturn(false);
    @@ -2313,7 +2315,7 @@ private TopologyDescription.Sink addSink(final String sinkName,
     
             topology.addSink(sinkName, sinkTopic, null, null, null, parentNames);
             final TopologyDescription.Sink expectedSinkNode =
    -            new InternalTopologyBuilder.Sink(sinkName, sinkTopic);
    +            new InternalTopologyBuilder.Sink<>(sinkName, sinkTopic);
     
             for (final TopologyDescription.Node parent : parents) {
                 ((InternalTopologyBuilder.AbstractNode) parent).addSuccessor(expectedSinkNode);
    @@ -2351,6 +2353,63 @@ private void addGlobalStoreToTopologyAndExpectedDescription(final String globalS
             expectedDescription.addGlobalStore(expectedGlobalStore);
         }
     
    +    @Test
    +    public void readOnlyStateStoresShouldHaveTheirOwnSubTopology() {
    +        final String sourceName = "source";
    +        final String storeName = "store";
    +        final String topicName = "topic";
    +        final String processorName = "processor";
    +
    +        final KeyValueStoreBuilder storeBuilder = mock(KeyValueStoreBuilder.class);
    +        when(storeBuilder.name()).thenReturn(storeName);
    +        topology.addReadOnlyStateStore(
    +                storeBuilder,
    +                sourceName,
    +                null,
    +                null,
    +                null,
    +                topicName,
    +                processorName,
    +                new MockProcessorSupplier<>());
    +
    +        final TopologyDescription.Source expectedSource = new InternalTopologyBuilder.Source(sourceName, Sets.newSet(topicName), null);
    +        final TopologyDescription.Processor expectedProcessor = new InternalTopologyBuilder.Processor(processorName, Sets.newSet(storeName));
    +
    +        ((InternalTopologyBuilder.AbstractNode) expectedSource).addSuccessor(expectedProcessor);
    +        ((InternalTopologyBuilder.AbstractNode) expectedProcessor).addPredecessor(expectedSource);
    +
    +        final Set allNodes = new HashSet<>();
    +        allNodes.add(expectedSource);
    +        allNodes.add(expectedProcessor);
    +        expectedDescription.addSubtopology(new SubtopologyDescription(0, allNodes));
    +
    +        assertThat(topology.describe(), equalTo(expectedDescription));
    +        assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode()));
    +    }
    +
    +    @Test
    +    public void readOnlyStateStoresShouldNotLog() {
    +        final String sourceName = "source";
    +        final String storeName = "store";
    +        final String topicName = "topic";
    +        final String processorName = "processor";
    +
    +        final KeyValueStoreBuilder storeBuilder = mock(KeyValueStoreBuilder.class);
    +        when(storeBuilder.name()).thenReturn(storeName);
    +        topology.addReadOnlyStateStore(
    +                storeBuilder,
    +                sourceName,
    +                null,
    +                null,
    +                null,
    +                topicName,
    +                processorName,
    +                new MockProcessorSupplier<>());
    +
    +        final StoreFactory stateStoreFactory = topology.internalTopologyBuilder.stateStores().get(storeName);
    +        assertThat(stateStoreFactory.loggingEnabled(), equalTo(false));
    +    }
    +
         private TopologyConfig overrideDefaultStore(final String defaultStore) {
             final Properties topologyOverrides = new Properties();
             // change default store as in-memory
    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 d79631f8f7..20199267ec 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
    @@ -123,8 +123,8 @@ public class EosIntegrationTest {
         public Timeout globalTimeout = Timeout.seconds(600);
         private static final Logger LOG = LoggerFactory.getLogger(EosIntegrationTest.class);
         private static final int NUM_BROKERS = 3;
    -    private static final int MAX_POLL_INTERVAL_MS = 5 * 1000;
    -    private static final int MAX_WAIT_TIME_MS = 60 * 1000;
    +    private static final int MAX_POLL_INTERVAL_MS = 30_1000;
    +    private static final int MAX_WAIT_TIME_MS = 120_1000;
     
         public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(
             NUM_BROKERS,
    @@ -189,6 +189,7 @@ public static Collection data() {
         public void createTopics() throws Exception {
             applicationId = "appId-" + TEST_NUMBER.getAndIncrement();
             CLUSTER.deleteTopicsAndWait(
    +            60_000L,
                 SINGLE_PARTITION_INPUT_TOPIC, MULTI_PARTITION_INPUT_TOPIC,
                 SINGLE_PARTITION_THROUGH_TOPIC, MULTI_PARTITION_THROUGH_TOPIC,
                 SINGLE_PARTITION_OUTPUT_TOPIC, MULTI_PARTITION_OUTPUT_TOPIC);
    @@ -403,7 +404,7 @@ public void shouldNotViolateEosIfOneTaskFails() throws Exception {
             // -> the failure only kills one thread
             // after fail over, we should read 40 committed records (even if 50 record got written)
     
    -        try (final KafkaStreams streams = getKafkaStreams("dummy", false, "appDir", 2, eosConfig, MAX_POLL_INTERVAL_MS)) {
    +        try (final KafkaStreams streams = getKafkaStreams("dummy", false, "appDir", 2, eosConfig)) {
                 startApplicationAndWaitUntilRunning(streams);
     
                 final List> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L);
    @@ -511,7 +512,7 @@ public void shouldNotViolateEosIfOneTaskFailsWithState() throws Exception {
     
             // We need more processing time under "with state" situation, so increasing the max.poll.interval.ms
             // to avoid unexpected rebalance during test, which will cause unexpected fail over triggered
    -        try (final KafkaStreams streams = getKafkaStreams("dummy", true, "appDir", 2, eosConfig, 3 * MAX_POLL_INTERVAL_MS)) {
    +        try (final KafkaStreams streams = getKafkaStreams("dummy", true, "appDir", 2, eosConfig)) {
                 startApplicationAndWaitUntilRunning(streams);
     
                 final List> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L);
    @@ -624,12 +625,12 @@ public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances() th
             // -> the stall only affects one thread and should trigger a rebalance
             // after rebalancing, we should read 40 committed records (even if 50 record got written)
             //
    -        // afterwards, the "stalling" thread resumes, and another rebalance should get triggered
    +        // afterward, the "stalling" thread resumes, and another rebalance should get triggered
             // we write the remaining 20 records and verify to read 60 result records
     
             try (
    -            final KafkaStreams streams1 = getKafkaStreams("streams1", false, "appDir1", 1, eosConfig, MAX_POLL_INTERVAL_MS);
    -            final KafkaStreams streams2 = getKafkaStreams("streams2", false, "appDir2", 1, eosConfig, MAX_POLL_INTERVAL_MS)
    +            final KafkaStreams streams1 = getKafkaStreams("streams1", false, "appDir1", 1, eosConfig);
    +            final KafkaStreams streams2 = getKafkaStreams("streams2", false, "appDir2", 1, eosConfig)
             ) {
                 startApplicationAndWaitUntilRunning(streams1);
                 startApplicationAndWaitUntilRunning(streams2);
    @@ -778,7 +779,7 @@ public void shouldWriteLatestOffsetsToCheckpointOnShutdown() throws Exception {
             final List> writtenData = prepareData(0L, 10, 0L, 1L);
             final List> expectedResult = computeExpectedResult(writtenData);
     
    -        try (final KafkaStreams streams = getKafkaStreams("streams", true, "appDir", 1, eosConfig, MAX_POLL_INTERVAL_MS)) {
    +        try (final KafkaStreams streams = getKafkaStreams("streams", true, "appDir", 1, eosConfig)) {
                 writeInputData(writtenData);
     
                 startApplicationAndWaitUntilRunning(streams);
    @@ -1004,8 +1005,7 @@ private KafkaStreams getKafkaStreams(final String dummyHostName,
                                              final boolean withState,
                                              final String appDir,
                                              final int numberOfStreamsThreads,
    -                                         final String eosConfig,
    -                                         final int maxPollIntervalMs) {
    +                                         final String eosConfig) {
             commitRequested = new AtomicInteger(0);
             errorInjected = new AtomicBoolean(false);
             stallInjected = new AtomicBoolean(false);
    @@ -1112,9 +1112,8 @@ public void close() { }
             properties.put(StreamsConfig.producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), (int) commitIntervalMs);
             properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.METADATA_MAX_AGE_CONFIG), "1000");
             properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "earliest");
    -        properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG), maxPollIntervalMs);
    -        properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), maxPollIntervalMs - 1);
    -        properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), maxPollIntervalMs);
    +        properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), MAX_POLL_INTERVAL_MS - 1);
    +        properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), MAX_POLL_INTERVAL_MS);
             properties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
             properties.put(StreamsConfig.STATE_DIR_CONFIG, stateTmpDir + appDir);
             properties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, dummyHostName + ":2142");
    diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java
    index 011090c152..bac9ae37eb 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java
    @@ -939,7 +939,7 @@ public void close() {}
             final Properties properties = new Properties();
             properties.put(StreamsConfig.CLIENT_ID_CONFIG, appDir);
             properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingGuarantee);
    -        final long commitInterval = Duration.ofMinutes(1L).toMillis();
    +        final long commitInterval = Duration.ofMinutes(5L).toMillis();
             properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, commitInterval);
             properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.METADATA_MAX_AGE_CONFIG), Duration.ofSeconds(1L).toMillis());
             properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "earliest");
    diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKStreamIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKStreamIntegrationTest.java
    index 1d9a77b5bf..10ab37cee0 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKStreamIntegrationTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKStreamIntegrationTest.java
    @@ -27,6 +27,7 @@
     import org.apache.kafka.streams.KeyValue;
     import org.apache.kafka.streams.StreamsBuilder;
     import org.apache.kafka.streams.StreamsConfig;
    +import org.apache.kafka.streams.StreamsConfig.InternalConfig;
     import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
     import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
     import org.apache.kafka.streams.kstream.JoinWindows;
    @@ -99,6 +100,7 @@ public void before(final TestInfo testInfo) throws IOException {
             final String safeTestName = safeUniqueTestName(testInfo);
             streamsConfig = getStreamsConfig(safeTestName);
             streamsConfig.put(StreamsConfig.STATE_DIR_CONFIG, stateDirBasePath);
    +        streamsConfig.put(InternalConfig.EMIT_INTERVAL_MS_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, 0L);
         }
     
         @AfterEach
    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 812da30074..b41e0c3d29 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
    @@ -28,6 +28,7 @@
     import org.apache.kafka.common.serialization.IntegerDeserializer;
     import org.apache.kafka.common.serialization.IntegerSerializer;
     import org.apache.kafka.common.serialization.Serdes;
    +import org.apache.kafka.common.serialization.StringDeserializer;
     import org.apache.kafka.common.utils.Bytes;
     import org.apache.kafka.common.utils.MockTime;
     import org.apache.kafka.common.utils.Utils;
    @@ -155,8 +156,8 @@ private Properties props(final Properties extraProperties) {
             streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
             streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
             streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).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.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class);
    +        streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class);
             streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
             streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
             streamsConfiguration.putAll(extraProperties);
    @@ -196,8 +197,8 @@ public void shouldRestoreNullRecord() throws Exception {
             final Properties streamsConfiguration = StreamsTestUtils.getStreamsConfig(
                     applicationId,
                     CLUSTER.bootstrapServers(),
    -                Serdes.Integer().getClass().getName(),
    -                Serdes.ByteArray().getClass().getName(),
    +                Serdes.IntegerSerde.class.getName(),
    +                Serdes.BytesSerde.class.getName(),
                     props);
     
             CLUSTER.createTopics(inputTopic);
    @@ -249,7 +250,63 @@ public void shouldRestoreNullRecord() throws Exception {
     
         @ParameterizedTest
         @MethodSource("parameters")
    -    public void shouldRestoreStateFromSourceTopic(final boolean stateUpdaterEnabled) throws Exception {
    +    public void shouldRestoreStateFromSourceTopicForReadOnlyStore(final boolean stateUpdaterEnabled) throws Exception {
    +        final AtomicInteger numReceived = new AtomicInteger(0);
    +        final Topology topology = new Topology();
    +
    +        final Properties props = props(stateUpdaterEnabled);
    +
    +        // 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(inputStream, 0);
    +        setCommittedOffset(inputStream, offsetLimitDelta);
    +
    +        final StateDirectory stateDirectory = new StateDirectory(new StreamsConfig(props), new MockTime(), true, false);
    +        // note here the checkpointed offset is the last processed record's offset, so without control message we should write this offset - 1
    +        new OffsetCheckpoint(new File(stateDirectory.getOrCreateDirectoryForTask(new TaskId(0, 0)), ".checkpoint"))
    +            .write(Collections.singletonMap(new TopicPartition(inputStream, 0), (long) offsetCheckpointed - 1));
    +        new OffsetCheckpoint(new File(stateDirectory.getOrCreateDirectoryForTask(new TaskId(0, 1)), ".checkpoint"))
    +            .write(Collections.singletonMap(new TopicPartition(inputStream, 1), (long) offsetCheckpointed - 1));
    +
    +        final CountDownLatch startupLatch = new CountDownLatch(1);
    +        final CountDownLatch shutdownLatch = new CountDownLatch(1);
    +
    +        topology.addReadOnlyStateStore(
    +            Stores.keyValueStoreBuilder(
    +                Stores.persistentKeyValueStore("store"),
    +                new Serdes.IntegerSerde(),
    +                new Serdes.StringSerde()
    +            ),
    +            "readOnlySource",
    +            new IntegerDeserializer(),
    +            new StringDeserializer(),
    +            inputStream,
    +            "readOnlyProcessor",
    +            () -> new ReadOnlyStoreProcessor(numReceived, offsetLimitDelta, shutdownLatch)
    +        );
    +
    +        kafkaStreams = new KafkaStreams(topology, props);
    +        kafkaStreams.setStateListener((newState, oldState) -> {
    +            if (newState == KafkaStreams.State.RUNNING && oldState == KafkaStreams.State.REBALANCING) {
    +                startupLatch.countDown();
    +            }
    +        });
    +
    +        final AtomicLong restored = new AtomicLong(0);
    +        kafkaStreams.setGlobalStateRestoreListener(new TrackingStateRestoreListener(restored));
    +        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));
    +    }
    +
    +    @ParameterizedTest
    +    @MethodSource("parameters")
    +    public void shouldRestoreStateFromSourceTopicForGlobalTable(final boolean stateUpdaterEnabled) throws Exception {
             final AtomicInteger numReceived = new AtomicInteger(0);
             final StreamsBuilder builder = new StreamsBuilder();
     
    @@ -265,9 +322,9 @@ public void shouldRestoreStateFromSourceTopic(final boolean stateUpdaterEnabled)
             final StateDirectory stateDirectory = new StateDirectory(new StreamsConfig(props), new MockTime(), true, false);
             // note here the checkpointed offset is the last processed record's offset, so without control message we should write this offset - 1
             new OffsetCheckpoint(new File(stateDirectory.getOrCreateDirectoryForTask(new TaskId(0, 0)), ".checkpoint"))
    -                .write(Collections.singletonMap(new TopicPartition(inputStream, 0), (long) offsetCheckpointed - 1));
    +            .write(Collections.singletonMap(new TopicPartition(inputStream, 0), (long) offsetCheckpointed - 1));
             new OffsetCheckpoint(new File(stateDirectory.getOrCreateDirectoryForTask(new TaskId(0, 1)), ".checkpoint"))
    -                .write(Collections.singletonMap(new TopicPartition(inputStream, 1), (long) offsetCheckpointed - 1));
    +            .write(Collections.singletonMap(new TopicPartition(inputStream, 1), (long) offsetCheckpointed - 1));
     
             final CountDownLatch startupLatch = new CountDownLatch(1);
             final CountDownLatch shutdownLatch = new CountDownLatch(1);
    @@ -288,22 +345,7 @@ public void shouldRestoreStateFromSourceTopic(final boolean stateUpdaterEnabled)
             });
     
             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.setGlobalStateRestoreListener(new TrackingStateRestoreListener(restored));
             kafkaStreams.start();
     
             assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
    @@ -332,9 +374,9 @@ public void shouldRestoreStateFromChangelogTopic(final boolean stateUpdaterEnabl
             final StateDirectory stateDirectory = new StateDirectory(new StreamsConfig(props), new MockTime(), true, false);
             // note here the checkpointed offset is the last processed record's offset, so without control message we should write this offset - 1
             new OffsetCheckpoint(new File(stateDirectory.getOrCreateDirectoryForTask(new TaskId(0, 0)), ".checkpoint"))
    -                .write(Collections.singletonMap(new TopicPartition(changelog, 0), (long) offsetCheckpointed - 1));
    +            .write(Collections.singletonMap(new TopicPartition(changelog, 0), (long) offsetCheckpointed - 1));
             new OffsetCheckpoint(new File(stateDirectory.getOrCreateDirectoryForTask(new TaskId(0, 1)), ".checkpoint"))
    -                .write(Collections.singletonMap(new TopicPartition(changelog, 1), (long) offsetCheckpointed - 1));
    +            .write(Collections.singletonMap(new TopicPartition(changelog, 1), (long) offsetCheckpointed - 1));
     
             final CountDownLatch startupLatch = new CountDownLatch(1);
             final CountDownLatch shutdownLatch = new CountDownLatch(1);
    @@ -355,22 +397,7 @@ public void shouldRestoreStateFromChangelogTopic(final boolean stateUpdaterEnabl
             });
     
             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.setGlobalStateRestoreListener(new TrackingStateRestoreListener(restored));
             kafkaStreams.start();
     
             assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
    @@ -386,10 +413,12 @@ public void shouldSuccessfullyStartWhenLoggingDisabled(final boolean stateUpdate
             final StreamsBuilder builder = new StreamsBuilder();
     
             final KStream stream = builder.stream(inputStream);
    -        stream.groupByKey()
    -                .reduce(
    -                        (value1, value2) -> value1 + value2,
    -                        Materialized.>as("reduce-store").withLoggingDisabled());
    +        stream
    +            .groupByKey()
    +            .reduce(
    +                Integer::sum,
    +                Materialized.>as("reduce-store").withLoggingDisabled()
    +            );
     
             final CountDownLatch startupLatch = new CountDownLatch(1);
             kafkaStreams = new KafkaStreams(builder.build(), props(stateUpdaterEnabled));
    @@ -821,4 +850,30 @@ private void waitForTransitionTo(final Set observed, final K
                 () -> "Client did not transition to " + state + " on time. Observed transitions: " + observed
             );
         }
    +
    +    private static class ReadOnlyStoreProcessor implements Processor {
    +        private final AtomicInteger numReceived;
    +        private final int offsetLimitDelta;
    +        private final CountDownLatch shutdownLatch;
    +        KeyValueStore store;
    +
    +        public ReadOnlyStoreProcessor(final AtomicInteger numReceived, final int offsetLimitDelta, final CountDownLatch shutdownLatch) {
    +            this.numReceived = numReceived;
    +            this.offsetLimitDelta = offsetLimitDelta;
    +            this.shutdownLatch = shutdownLatch;
    +        }
    +
    +        @Override
    +        public void init(final ProcessorContext context) {
    +            store = context.getStateStore("store");
    +        }
    +
    +        @Override
    +        public void process(final Record record) {
    +            store.put(record.key(), record.value());
    +            if (numReceived.incrementAndGet() == offsetLimitDelta * 2) {
    +                shutdownLatch.countDown();
    +            }
    +        }
    +    }
     }
    \ No newline at end of file
    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 3f3e9a243b..4232e1d74c 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
    @@ -24,6 +24,7 @@
     import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
     import org.apache.kafka.server.config.ConfigType;
     import org.apache.kafka.server.util.MockTime;
    +import org.apache.kafka.storage.internals.log.CleanerConfig;
     import org.apache.kafka.test.TestCondition;
     import org.apache.kafka.test.TestUtils;
     import org.slf4j.Logger;
    @@ -111,7 +112,7 @@ public void start() throws IOException {
             brokerConfig.put(KafkaConfig.ZkConnectProp(), zKConnectString());
             putIfAbsent(brokerConfig, KafkaConfig.ListenersProp(), "PLAINTEXT://localhost:" + DEFAULT_BROKER_PORT);
             putIfAbsent(brokerConfig, KafkaConfig.DeleteTopicEnableProp(), true);
    -        putIfAbsent(brokerConfig, KafkaConfig.LogCleanerDedupeBufferSizeProp(), 2 * 1024 * 1024L);
    +        putIfAbsent(brokerConfig, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, 2 * 1024 * 1024L);
             putIfAbsent(brokerConfig, KafkaConfig.GroupMinSessionTimeoutMsProp(), 0);
             putIfAbsent(brokerConfig, KafkaConfig.GroupInitialRebalanceDelayMsProp(), 0);
             putIfAbsent(brokerConfig, KafkaConfig.OffsetsTopicReplicationFactorProp(), (short) 1);
    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 4f1d8d3d42..7c5734d7b9 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
    @@ -1518,6 +1518,15 @@ public static class TrackingStateRestoreListener implements StateRestoreListener
             public final Map changelogToStartOffset = new ConcurrentHashMap<>();
             public final Map changelogToEndOffset = new ConcurrentHashMap<>();
             public final Map changelogToTotalNumRestored = new ConcurrentHashMap<>();
    +        private final AtomicLong restored;
    +
    +        public TrackingStateRestoreListener() {
    +            restored = null;
    +        }
    +
    +        public TrackingStateRestoreListener(final AtomicLong restored) {
    +            this.restored = restored;
    +        }
     
             @Override
             public void onRestoreStart(final TopicPartition topicPartition,
    @@ -1541,6 +1550,9 @@ public void onBatchRestored(final TopicPartition topicPartition,
             public void onRestoreEnd(final TopicPartition topicPartition,
                                      final String storeName,
                                      final long totalRestored) {
    +            if (restored != null) {
    +                restored.addAndGet(totalRestored);
    +            }
             }
     
             public long totalNumRestored() {
    diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java
    index cc8f964245..944f72d932 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java
    @@ -28,9 +28,8 @@
     import java.util.Map;
     
     import static org.junit.Assert.assertNotNull;
    -import static org.junit.Assert.assertTrue;
     import static org.junit.Assert.assertThrows;
    -
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     
     public class SessionWindowedDeserializerTest {
         private final SessionWindowedDeserializer sessionWindowedDeserializer = new SessionWindowedDeserializer<>(new StringDeserializer());
    @@ -41,7 +40,7 @@ public void testSessionWindowedDeserializerConstructor() {
             sessionWindowedDeserializer.configure(props, true);
             final Deserializer inner = sessionWindowedDeserializer.innerDeserializer();
             assertNotNull("Inner deserializer should be not null", inner);
    -        assertTrue("Inner deserializer type should be StringDeserializer", inner instanceof StringDeserializer);
    +        assertInstanceOf(StringDeserializer.class, inner, "Inner deserializer type should be StringDeserializer");
         }
     
         @Test
    @@ -49,7 +48,7 @@ public void shouldSetWindowedInnerClassDeserialiserThroughConfig() {
             props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName());
             final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>();
             deserializer.configure(props, false);
    -        assertTrue(deserializer.innerDeserializer() instanceof ByteArrayDeserializer);
    +        assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer());
         }
     
         @Test
    diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java
    index 2a560ed5ac..78b67643b2 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java
    @@ -28,8 +28,8 @@
     import java.util.Map;
     
     import static org.junit.Assert.assertNotNull;
    -import static org.junit.Assert.assertTrue;
     import static org.junit.Assert.assertThrows;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     
     public class SessionWindowedSerializerTest {
         private final SessionWindowedSerializer sessionWindowedSerializer = new SessionWindowedSerializer<>(Serdes.String().serializer());
    @@ -40,7 +40,7 @@ public void testSessionWindowedSerializerConstructor() {
             sessionWindowedSerializer.configure(props, true);
             final Serializer inner = sessionWindowedSerializer.innerSerializer();
             assertNotNull("Inner serializer should be not null", inner);
    -        assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer);
    +        assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer");
         }
     
         @Test
    @@ -48,7 +48,7 @@ public void shouldSetWindowedInnerClassSerialiserThroughConfig() {
             props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName());
             final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>();
             serializer.configure(props, false);
    -        assertTrue(serializer.innerSerializer() instanceof ByteArraySerializer);
    +        assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer());
         }
     
         @Test
    diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java
    index a0357636d9..f3b1b8323b 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java
    @@ -31,7 +31,7 @@
     import static org.hamcrest.core.Is.is;
     import static org.junit.Assert.assertNotNull;
     import static org.junit.Assert.assertThrows;
    -import static org.junit.Assert.assertTrue;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     
     public class TimeWindowedDeserializerTest {
         private final long windowSize = 5000000;
    @@ -43,7 +43,7 @@ public void testTimeWindowedDeserializerConstructor() {
             timeWindowedDeserializer.configure(props, true);
             final Deserializer inner = timeWindowedDeserializer.innerDeserializer();
             assertNotNull("Inner deserializer should be not null", inner);
    -        assertTrue("Inner deserializer type should be StringDeserializer", inner instanceof StringDeserializer);
    +        assertInstanceOf(StringDeserializer.class, inner, "Inner deserializer type should be StringDeserializer");
             assertThat(timeWindowedDeserializer.getWindowSize(), is(5000000L));
         }
     
    @@ -54,7 +54,7 @@ public void shouldSetWindowSizeAndWindowedInnerDeserialiserThroughConfigs() {
             final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>();
             deserializer.configure(props, false);
             assertThat(deserializer.getWindowSize(), is(500L));
    -        assertTrue(deserializer.innerDeserializer() instanceof ByteArrayDeserializer);
    +        assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer());
         }
     
         @Test
    diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java
    index b5e9754430..5b221f052b 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java
    @@ -29,7 +29,7 @@
     
     import static org.junit.Assert.assertNotNull;
     import static org.junit.Assert.assertThrows;
    -import static org.junit.Assert.assertTrue;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     
     public class TimeWindowedSerializerTest {
         private final TimeWindowedSerializer timeWindowedSerializer = new TimeWindowedSerializer<>(Serdes.String().serializer());
    @@ -40,7 +40,7 @@ public void testTimeWindowedSerializerConstructor() {
             timeWindowedSerializer.configure(props, true);
             final Serializer inner = timeWindowedSerializer.innerSerializer();
             assertNotNull("Inner serializer should be not null", inner);
    -        assertTrue("Inner serializer type should be StringSerializer", inner instanceof StringSerializer);
    +        assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer");
         }
     
         @Test
    @@ -48,7 +48,7 @@ public void shouldSetWindowedInnerClassSerialiserThroughConfig() {
             props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName());
             final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>();
             serializer.configure(props, false);
    -        assertTrue(serializer.innerSerializer() instanceof ByteArraySerializer);
    +        assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer());
         }
     
         @Test
    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
    index f12d0db660..4c7e97be40 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowedSerdesTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowedSerdesTest.java
    @@ -27,7 +27,7 @@
     import static org.hamcrest.MatcherAssert.assertThat;
     import static org.hamcrest.Matchers.equalTo;
     import static org.junit.Assert.assertThrows;
    -import static org.junit.Assert.assertTrue;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     
     public class WindowedSerdesTest {
     
    @@ -36,19 +36,19 @@ public class WindowedSerdesTest {
         @Test
         public void shouldWrapForTimeWindowedSerde() {
             final Serde> serde = WindowedSerdes.timeWindowedSerdeFrom(String.class, Long.MAX_VALUE);
    -        assertTrue(serde.serializer() instanceof TimeWindowedSerializer);
    -        assertTrue(serde.deserializer() instanceof TimeWindowedDeserializer);
    -        assertTrue(((TimeWindowedSerializer) serde.serializer()).innerSerializer() instanceof StringSerializer);
    -        assertTrue(((TimeWindowedDeserializer) serde.deserializer()).innerDeserializer() instanceof StringDeserializer);
    +        assertInstanceOf(TimeWindowedSerializer.class, serde.serializer());
    +        assertInstanceOf(TimeWindowedDeserializer.class, serde.deserializer());
    +        assertInstanceOf(StringSerializer.class, ((TimeWindowedSerializer) serde.serializer()).innerSerializer());
    +        assertInstanceOf(StringDeserializer.class, ((TimeWindowedDeserializer) serde.deserializer()).innerDeserializer());
         }
     
         @Test
         public void shouldWrapForSessionWindowedSerde() {
             final Serde> serde = WindowedSerdes.sessionWindowedSerdeFrom(String.class);
    -        assertTrue(serde.serializer() instanceof SessionWindowedSerializer);
    -        assertTrue(serde.deserializer() instanceof SessionWindowedDeserializer);
    -        assertTrue(((SessionWindowedSerializer) serde.serializer()).innerSerializer() instanceof StringSerializer);
    -        assertTrue(((SessionWindowedDeserializer) serde.deserializer()).innerDeserializer() instanceof StringDeserializer);
    +        assertInstanceOf(SessionWindowedSerializer.class, serde.serializer());
    +        assertInstanceOf(SessionWindowedDeserializer.class, serde.deserializer());
    +        assertInstanceOf(StringSerializer.class, ((SessionWindowedSerializer) serde.serializer()).innerSerializer());
    +        assertInstanceOf(StringDeserializer.class, ((SessionWindowedDeserializer) serde.deserializer()).innerDeserializer());
         }
     
         @Test
    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 1106e914cf..3517dfc651 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
    @@ -74,6 +74,7 @@
     import static org.junit.Assert.assertNull;
     import static org.junit.Assert.assertThrows;
     import static org.junit.Assert.assertTrue;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     
     public class InternalStreamsBuilderTest {
     
    @@ -1186,40 +1187,40 @@ public void shouldSetUseVersionedSemanticsOnTableForeignSelfJoin() {
     
         private void verifyVersionedSemantics(final TableFilterNode filterNode, final boolean expectedValue) {
             final ProcessorSupplier processorSupplier = filterNode.processorParameters().processorSupplier();
    -        assertTrue(processorSupplier instanceof KTableFilter);
    +        assertInstanceOf(KTableFilter.class, processorSupplier);
             final KTableFilter tableFilter = (KTableFilter) processorSupplier;
             assertEquals(expectedValue, tableFilter.isUseVersionedSemantics());
         }
     
         private void verifyVersionedSemantics(final TableRepartitionMapNode repartitionMapNode, final boolean expectedValue) {
             final ProcessorSupplier processorSupplier = repartitionMapNode.processorParameters().processorSupplier();
    -        assertTrue(processorSupplier instanceof KTableRepartitionMap);
    +        assertInstanceOf(KTableRepartitionMap.class, processorSupplier);
             final KTableRepartitionMap repartitionMap = (KTableRepartitionMap) processorSupplier;
             assertEquals(expectedValue, repartitionMap.isUseVersionedSemantics());
         }
     
         private void verifyVersionedSemantics(final KTableKTableJoinNode joinNode, final boolean expectedValueLeft, final boolean expectedValueRight) {
             final ProcessorSupplier thisProcessorSupplier = joinNode.thisProcessorParameters().processorSupplier();
    -        assertTrue(thisProcessorSupplier instanceof KTableKTableAbstractJoin);
    +        assertInstanceOf(KTableKTableAbstractJoin.class, thisProcessorSupplier);
             final KTableKTableAbstractJoin thisJoin = (KTableKTableAbstractJoin) thisProcessorSupplier;
             assertEquals(expectedValueLeft, thisJoin.isUseVersionedSemantics());
     
             final ProcessorSupplier otherProcessorSupplier = joinNode.otherProcessorParameters().processorSupplier();
    -        assertTrue(otherProcessorSupplier instanceof KTableKTableAbstractJoin);
    +        assertInstanceOf(KTableKTableAbstractJoin.class, otherProcessorSupplier);
             final KTableKTableAbstractJoin otherJoin = (KTableKTableAbstractJoin) otherProcessorSupplier;
             assertEquals(expectedValueRight, otherJoin.isUseVersionedSemantics());
         }
     
         private void verifyVersionedSemantics(final ForeignJoinSubscriptionSendNode joinThisNode, final boolean expectedValue) {
             final ProcessorSupplier thisProcessorSupplier = joinThisNode.processorParameters().processorSupplier();
    -        assertTrue(thisProcessorSupplier instanceof SubscriptionSendProcessorSupplier);
    +        assertInstanceOf(SubscriptionSendProcessorSupplier.class, thisProcessorSupplier);
             final SubscriptionSendProcessorSupplier joinThis = (SubscriptionSendProcessorSupplier) thisProcessorSupplier;
             assertEquals(expectedValue, joinThis.isUseVersionedSemantics());
         }
     
         private void verifyVersionedSemantics(final ForeignTableJoinNode joinOtherNode, final boolean expectedValue) {
             final ProcessorSupplier otherProcessorSupplier = joinOtherNode.processorParameters().processorSupplier();
    -        assertTrue(otherProcessorSupplier instanceof ForeignTableJoinProcessorSupplier);
    +        assertInstanceOf(ForeignTableJoinProcessorSupplier.class, otherProcessorSupplier);
             final ForeignTableJoinProcessorSupplier joinThis = (ForeignTableJoinProcessorSupplier) otherProcessorSupplier;
             assertEquals(expectedValue, joinThis.isUseVersionedSemantics());
         }
    diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java
    index 2a3042e2fc..9ea08636e5 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java
    @@ -34,7 +34,7 @@
     import java.util.Arrays;
     import java.util.Collections;
     
    -import static org.junit.Assert.assertTrue;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     import static org.mockito.Mockito.inOrder;
     import static org.mockito.Mockito.mock;
     import static org.mockito.Mockito.never;
    @@ -129,6 +129,6 @@ public void shouldGetFlatTransformProcessor() {
     
             final Processor processor = processorSupplier.get();
     
    -        assertTrue(processor instanceof KStreamFlatTransformProcessor);
    +        assertInstanceOf(KStreamFlatTransformProcessor.class, processor);
         }
     }
    diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java
    index 988eab92bf..4b9e2f82d9 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java
    @@ -16,7 +16,7 @@
      */
     package org.apache.kafka.streams.kstream.internals;
     
    -import static org.junit.Assert.assertTrue;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     import static org.mockito.Mockito.inOrder;
     import static org.mockito.Mockito.mock;
     import static org.mockito.Mockito.never;
    @@ -129,6 +129,6 @@ public void shouldGetFlatTransformValuesProcessor() {
     
             final Processor processor = processorSupplier.get();
     
    -        assertTrue(processor instanceof KStreamFlatTransformValuesProcessor);
    +        assertInstanceOf(KStreamFlatTransformValuesProcessor.class, processor);
         }
     }
    diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
    index 156b553455..fd36b241b2 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
    @@ -436,6 +436,184 @@ public void testRightNonJoinedRecordsAreNeverEmittedByTheRightProcessor() {
             }
         }
     
    +    @Test
    +    public void testLeftJoinedRecordsWithZeroAfterAreEmitted() {
    +        final StreamsBuilder builder = new StreamsBuilder();
    +
    +        final int[] expectedKeys = new int[] {0, 1, 2, 3};
    +
    +        final KStream stream1;
    +        final KStream stream2;
    +        final KStream joined;
    +        final MockApiProcessorSupplier supplier = new MockApiProcessorSupplier<>();
    +        stream1 = builder.stream(topic1, consumed);
    +        stream2 = builder.stream(topic2, consumed);
    +        
    +        joined = stream1.leftJoin(
    +            stream2,
    +            MockValueJoiner.TOSTRING_JOINER,
    +            JoinWindows.ofTimeDifferenceWithNoGrace(ofMillis(100)).after(ZERO),
    +            StreamJoined.with(Serdes.Integer(),
    +                Serdes.String(),
    +                Serdes.String())
    +        );
    +        joined.process(supplier);
    +
    +        final Collection> copartitionGroups =
    +            TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
    +
    +        assertEquals(1, copartitionGroups.size());
    +        assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
    +
    +        try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) {
    +            final TestInputTopic inputTopic1 =
    +                    driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO);
    +            final TestInputTopic inputTopic2 =
    +                    driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO);
    +            final MockApiProcessor processor = supplier.theCapturedProcessor();
    +
    +            processor.init(null);
    +            
    +            // push four items with increasing timestamps to the primary stream; the other window is empty; 
    +            // this should emit the first three left-joined items;
    +            // A3 is not triggered yet
    +            // w1 = {}
    +            // w2 = {}
    +            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
    +            //     w2 = {}
    +            long time = 1000L;
    +            for (int i = 0; i < expectedKeys.length; i++) {
    +                inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i], time + i);
    +            }
    +            processor.checkAndClearProcessResult(
    +                    new KeyValueTimestamp<>(0, "A0+null", 1000L),
    +                    new KeyValueTimestamp<>(1, "A1+null", 1001L),
    +                    new KeyValueTimestamp<>(2, "A2+null", 1002L)
    +            );
    +            
    +            // push four items smaller timestamps (out of window) to the secondary stream; 
    +            // this should produce four joined items
    +            // w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
    +            // w2 = {}
    +            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
    +            //     w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999) }
    +            time = 1000L - 1L;
    +            for (final int expectedKey : expectedKeys) {
    +                inputTopic2.pipeInput(expectedKey, "a" + expectedKey, time);
    +            }
    +            processor.checkAndClearProcessResult(
    +                    new KeyValueTimestamp<>(0, "A0+a0", 1000L),
    +                    new KeyValueTimestamp<>(1, "A1+a1", 1001L),
    +                    new KeyValueTimestamp<>(2, "A2+a2", 1002L),
    +                    new KeyValueTimestamp<>(3, "A3+a3", 1003L)
    +            );
    +
    +            // push four items with increased timestamps to the secondary stream; 
    +            // this should produce four joined item
    +            // w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
    +            // w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999) }
    +            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
    +            //     w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
    +            //            0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000) }
    +            time += 1L;
    +            for (final int expectedKey : expectedKeys) {
    +                inputTopic2.pipeInput(expectedKey, "b" + expectedKey, time);
    +            }
    +            processor.checkAndClearProcessResult(
    +                    new KeyValueTimestamp<>(0, "A0+b0", 1000L),
    +                    new KeyValueTimestamp<>(1, "A1+b1", 1001L),
    +                    new KeyValueTimestamp<>(2, "A2+b2", 1002L),
    +                    new KeyValueTimestamp<>(3, "A3+b3", 1003L)
    +            );
    +
    +            // push four items with increased timestamps to the secondary stream; 
    +            // this should produce only three joined items;
    +            // c0 arrives too late to be joined with A0
    +            // w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
    +            // w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
    +            //        0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000) }
    +            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
    +            //     w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
    +            //            0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
    +            //            0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001) }
    +            time += 1L;
    +            for (final int expectedKey : expectedKeys) {
    +                inputTopic2.pipeInput(expectedKey, "c" + expectedKey, time);
    +            }
    +            processor.checkAndClearProcessResult(
    +                    new KeyValueTimestamp<>(1, "A1+c1", 1001L),
    +                    new KeyValueTimestamp<>(2, "A2+c2", 1002L),
    +                    new KeyValueTimestamp<>(3, "A3+c3", 1003L)
    +            );
    +
    +            // push four items with increased timestamps to the secondary stream;
    +            // this should produce only two joined items;
    +            // d0 and d1 arrive too late to be joined with A0 and A1
    +            // w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
    +            // w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
    +            //        0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
    +            //        0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001) }
    +            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
    +            //     w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
    +            //            0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
    +            //            0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001),
    +            //            0:d0 (ts: 1002), 1:d1 (ts: 1002), 2:d2 (ts: 1002), 3:d3 (ts: 1002) }
    +            time += 1L;
    +            for (final int expectedKey : expectedKeys) {
    +                inputTopic2.pipeInput(expectedKey, "d" + expectedKey, time);
    +            }
    +            processor.checkAndClearProcessResult(
    +                    new KeyValueTimestamp<>(2, "A2+d2", 1002L),
    +                    new KeyValueTimestamp<>(3, "A3+d3", 1003L)
    +            );
    +
    +            // push four items with increased timestamps to the secondary stream; 
    +            // this should produce one joined item;
    +            // only e3 can be joined with A3;
    +            // e0, e1 and e2 arrive too late to be joined with A0, A1 and A2
    +            // w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
    +            // w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
    +            //        0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
    +            //        0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001),
    +            //        0:d0 (ts: 1002), 1:d1 (ts: 1002), 2:d2 (ts: 1002), 3:d3 (ts: 1002) }
    +            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
    +            //     w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
    +            //            0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
    +            //            0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001),
    +            //            0:d0 (ts: 1002), 1:d1 (ts: 1002), 2:d2 (ts: 1002), 3:d3 (ts: 1002),
    +            //            0:e0 (ts: 1003), 1:e1 (ts: 1003), 2:e2 (ts: 1003), 3:e3 (ts: 1003) }
    +            time += 1L;
    +            for (final int expectedKey : expectedKeys) {
    +                inputTopic2.pipeInput(expectedKey, "e" + expectedKey, time);
    +            }
    +            processor.checkAndClearProcessResult(
    +                new KeyValueTimestamp<>(3, "A3+e3", 1003L)
    +            );
    +
    +            // push four items with larger timestamps to the secondary stream;
    +            // no (non-)joined items can be produced
    +            // 
    +            // w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
    +            // w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
    +            //        0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
    +            //        0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001),
    +            //        0:d0 (ts: 1002), 1:d1 (ts: 1002), 2:d2 (ts: 1002), 3:d3 (ts: 1002),
    +            //        0:e0 (ts: 1003), 1:e1 (ts: 1003), 2:e2 (ts: 1003), 3:e3 (ts: 1003) }
    +            // --> w1 = { 0:A0 (ts: 1000), 1:A1 (ts: 1001), 2:A2 (ts: 1002), 3:A3 (ts: 1003) }
    +            //     w2 = { 0:a0 (ts: 999), 1:a1 (ts: 999), 2:a2 (ts: 999), 3:a3 (ts: 999),
    +            //            0:b0 (ts: 1000), 1:b1 (ts: 1000), 2:b2 (ts: 1000), 3:b3 (ts: 1000),
    +            //            0:c0 (ts: 1001), 1:c1 (ts: 1001), 2:c2 (ts: 1001), 3:c3 (ts: 1001),
    +            //            0:d0 (ts: 1002), 1:d1 (ts: 1002), 2:d2 (ts: 1002), 3:d3 (ts: 1002),
    +            //            0:e0 (ts: 1003), 1:e1 (ts: 1003), 2:e2 (ts: 1003), 3:e3 (ts: 1003),
    +            //            0:f0 (ts: 1100), 1:f1 (ts: 1100), 2:f2 (ts: 1100), 3:f3 (ts: 1100) }
    +            time = 1000 + 100L;
    +            for (final int expectedKey : expectedKeys) {
    +                inputTopic2.pipeInput(expectedKey, "f" + expectedKey, time);
    +            }
    +            processor.checkAndClearProcessResult();
    +        }
    +    }
    +
         @Test
         public void testLeftJoinWithInMemoryCustomSuppliers() {
             final JoinWindows joinWindows = JoinWindows.ofTimeDifferenceAndGrace(ofMillis(100L), ofMillis(0L));
    @@ -609,8 +787,9 @@ public void testOrdering() {
                 inputTopic1.pipeInput(1, "A1", 100L);
                 processor.checkAndClearProcessResult();
     
    -            // push one item to the other window that has a join; this should produce non-joined records with a closed window first, then
    -            // the joined records
    +            // push one item to the other window that has a join; 
    +            // this should produce the joined record first;
    +            // then non-joined record with a closed window
                 // by the time they were produced before
                 // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 100) }
                 // w2 = { }
    diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java
    index 099dc5b0c8..28a5f1488f 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java
    @@ -108,11 +108,11 @@ public void testOuterJoinDuplicatesWithFixDisabledOldApi() {
                 inputTopic2.pipeInput(1, "b1", 0L);
     
                 processor.checkAndClearProcessResult(
    -                    new KeyValueTimestamp<>(0, "A0+null", 0L),
    -                    new KeyValueTimestamp<>(0, "A0-0+null", 0L),
    -                    new KeyValueTimestamp<>(0, "A0+a0", 0L),
    -                    new KeyValueTimestamp<>(0, "A0-0+a0", 0L),
    -                    new KeyValueTimestamp<>(1, "null+b1", 0L)
    +                new KeyValueTimestamp<>(0, "A0+null", 0L),
    +                new KeyValueTimestamp<>(0, "A0-0+null", 0L),
    +                new KeyValueTimestamp<>(0, "A0+a0", 0L),
    +                new KeyValueTimestamp<>(0, "A0-0+a0", 0L),
    +                new KeyValueTimestamp<>(1, "null+b1", 0L)
                 );
             }
         }
    @@ -438,13 +438,13 @@ public void testOrdering() {
                 inputTopic1.pipeInput(1, "A1", 100L);
                 processor.checkAndClearProcessResult();
     
    -            // push one item to the other window that has a join; this should produce non-joined records with a closed window first, then
    -            // the joined records
    -            // by the time they were produced before
    +            // push one item to the other window that has a join;
    +            // this should produce the not-joined record first;
    +            // then the joined record
                 // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 100) }
                 // w2 = { }
    -            // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) }
    -            // --> w2 = { 0:a0 (ts: 110) }
    +            // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 100) }
    +            // --> w2 = { 1:a1 (ts: 110) }
                 inputTopic2.pipeInput(1, "a1", 110L);
                 processor.checkAndClearProcessResult(
                     new KeyValueTimestamp<>(0, "A0+null", 0L),
    @@ -788,7 +788,7 @@ public void shouldNotEmitLeftJoinResultForAsymmetricBeforeWindow() {
                     new KeyValueTimestamp<>(1, "A1+null", 1L)
                 );
     
    -            // push one item to the other stream; this should not produce any items
    +            // push one item to the other stream; this should produce one right-join item
                 // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
                 // w2 = { 0:a0 (ts: 100), 1:a1 (ts: 102) }
                 // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
    @@ -841,7 +841,8 @@ public void shouldNotEmitLeftJoinResultForAsymmetricAfterWindow() {
                 final MockApiProcessor processor = supplier.theCapturedProcessor();
                 long time = 0L;
     
    -            // push two items to the primary stream; the other window is empty; this should not produce any item
    +            // push two items to the primary stream; the other window is empty; 
    +            // this should produce one left-joined item
                 // w1 = {}
                 // w2 = {}
                 // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
    @@ -849,7 +850,9 @@ public void shouldNotEmitLeftJoinResultForAsymmetricAfterWindow() {
                 for (int i = 0; i < 2; i++) {
                     inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i], time + i);
                 }
    -            processor.checkAndClearProcessResult();
    +            processor.checkAndClearProcessResult(
    +                new KeyValueTimestamp<>(0, "A0+null", 0L)
    +            );
     
                 // push one item to the other stream; this should produce one full-join item
                 // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
    @@ -863,7 +866,8 @@ public void shouldNotEmitLeftJoinResultForAsymmetricAfterWindow() {
                     new KeyValueTimestamp<>(1, "A1+a1", 1L)
                 );
     
    -            // push one item to the other stream; this should produce one left-join item
    +            // push one item to the other stream;
    +            // this should not produce any item
                 // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
                 // w2 = { 1:a1 (ts: 1) }
                 // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
    @@ -871,9 +875,7 @@ public void shouldNotEmitLeftJoinResultForAsymmetricAfterWindow() {
                 time += 100;
                 inputTopic2.pipeInput(expectedKeys[2], "a" + expectedKeys[2], time);
     
    -            processor.checkAndClearProcessResult(
    -                new KeyValueTimestamp<>(0, "A0+null", 0L)
    -            );
    +            processor.checkAndClearProcessResult();
     
                 // push one item to the other stream; this should not produce any item
                 // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
    @@ -884,11 +886,12 @@ public void shouldNotEmitLeftJoinResultForAsymmetricAfterWindow() {
     
                 processor.checkAndClearProcessResult();
     
    -            // push one item to the first stream; this should produce one full-join item
    +            // push one item to the first stream;
    +            // this should produce one inner-join item;
                 // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1) }
                 // w2 = { 1:a1 (ts: 1), 2:a2 (ts: 101), 3:a3 (ts: 101) }
                 // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 1), 2:A2 (ts: 201) }
    -            // --> w2 = { 1:a1 (ts: 1), 2:a2 (ts: 101), 3:a3 (ts: 101 }
    +            // --> w2 = { 1:a1 (ts: 1), 2:a2 (ts: 101), 3:a3 (ts: 101) }
                 time += 100;
                 inputTopic1.pipeInput(expectedKeys[2], "A" + expectedKeys[2], time);
     
    diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignTableJoinProcessorSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignTableJoinProcessorSupplierTest.java
    index f4f35e6ff0..c292dd2e34 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignTableJoinProcessorSupplierTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignTableJoinProcessorSupplierTest.java
    @@ -379,4 +379,4 @@ SubscriptionResponseWrapper> processor(final KTableValueGetterSupplier(valueGetterSupplier);
             return supplier.get();
         }
    -}
    \ No newline at end of file
    +}
    diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplierTest.java
    index 4c26efe236..b32c51a3ba 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplierTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplierTest.java
    @@ -17,6 +17,7 @@
     
     package org.apache.kafka.streams.kstream.internals.foreignkeyjoin;
     
    +import org.apache.kafka.common.MetricName;
     import org.apache.kafka.common.serialization.StringSerializer;
     import org.apache.kafka.streams.kstream.ValueJoiner;
     import org.apache.kafka.streams.kstream.internals.KTableValueGetter;
    @@ -25,17 +26,23 @@
     import org.apache.kafka.streams.processor.api.Processor;
     import org.apache.kafka.streams.processor.api.ProcessorContext;
     import org.apache.kafka.streams.processor.api.Record;
    +import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
     import org.apache.kafka.streams.state.ValueAndTimestamp;
     import org.apache.kafka.streams.state.internals.Murmur3;
    +import org.apache.kafka.test.MockInternalNewProcessorContext;
     import org.junit.Test;
     
     import java.util.HashMap;
     import java.util.List;
     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.hamcrest.CoreMatchers.is;
     import static org.hamcrest.MatcherAssert.assertThat;
     import static org.hamcrest.collection.IsEmptyCollection.empty;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertNotEquals;
     
     public class ResponseJoinProcessorSupplierTest {
         private static final StringSerializer STRING_SERIALIZER = new StringSerializer();
    @@ -88,7 +95,7 @@ public void shouldNotForwardWhenHashDoesNotMatch() {
                     leftJoin
                 );
             final Processor, String, String> processor = processorSupplier.get();
    -        final org.apache.kafka.streams.processor.api.MockProcessorContext context = new org.apache.kafka.streams.processor.api.MockProcessorContext<>();
    +        final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>();
             processor.init(context);
             context.setRecordMetadata("topic", 0, 0);
     
    @@ -97,6 +104,10 @@ public void shouldNotForwardWhenHashDoesNotMatch() {
             processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(oldHash, "rhsValue", 0), 0));
             final List> forwarded = context.forwarded();
             assertThat(forwarded, empty());
    +
    +        // test dropped-records sensors
    +        assertEquals(1.0, getDroppedRecordsTotalMetric(context));
    +        assertNotEquals(0.0, getDroppedRecordsRateMetric(context));
         }
     
         @Test
    @@ -113,7 +124,7 @@ public void shouldIgnoreUpdateWhenLeftHasBecomeNull() {
                     leftJoin
                 );
             final Processor, String, String> processor = processorSupplier.get();
    -        final MockProcessorContext context = new MockProcessorContext<>();
    +        final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>();
             processor.init(context);
             context.setRecordMetadata("topic", 0, 0);
     
    @@ -122,6 +133,10 @@ public void shouldIgnoreUpdateWhenLeftHasBecomeNull() {
             processor.process(new Record<>("lhs1", new SubscriptionResponseWrapper<>(hash, "rhsValue", 0), 0));
             final List> forwarded = context.forwarded();
             assertThat(forwarded, empty());
    +
    +        // test dropped-records sensors
    +        assertEquals(1.0, getDroppedRecordsTotalMetric(context));
    +        assertNotEquals(0.0, getDroppedRecordsRateMetric(context));
         }
     
         @Test
    @@ -138,7 +153,7 @@ public void shouldForwardWhenHashMatches() {
                     leftJoin
                 );
             final Processor, String, String> processor = processorSupplier.get();
    -        final MockProcessorContext context = new MockProcessorContext<>();
    +        final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>();
             processor.init(context);
             context.setRecordMetadata("topic", 0, 0);
     
    @@ -164,7 +179,7 @@ public void shouldEmitTombstoneForInnerJoinWhenRightIsNull() {
                     leftJoin
                 );
             final Processor, String, String> processor = processorSupplier.get();
    -        final MockProcessorContext context = new MockProcessorContext<>();
    +        final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>();
             processor.init(context);
             context.setRecordMetadata("topic", 0, 0);
     
    @@ -190,7 +205,7 @@ public void shouldEmitResultForLeftJoinWhenRightIsNull() {
                     leftJoin
                 );
             final Processor, String, String> processor = processorSupplier.get();
    -        final MockProcessorContext context = new MockProcessorContext<>();
    +        final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>();
             processor.init(context);
             context.setRecordMetadata("topic", 0, 0);
     
    @@ -216,7 +231,7 @@ public void shouldEmitTombstoneForLeftJoinWhenRightIsNullAndLeftIsNull() {
                     leftJoin
                 );
             final Processor, String, String> processor = processorSupplier.get();
    -        final MockProcessorContext context = new MockProcessorContext<>();
    +        final MockInternalNewProcessorContext context = new MockInternalNewProcessorContext<>();
             processor.init(context);
             context.setRecordMetadata("topic", 0, 0);
     
    @@ -227,4 +242,33 @@ public void shouldEmitTombstoneForLeftJoinWhenRightIsNullAndLeftIsNull() {
             assertThat(forwarded.size(), is(1));
             assertThat(forwarded.get(0).record(), is(new Record<>("lhs1", null, 0)));
         }
    +
    +    private Object getDroppedRecordsTotalMetric(final InternalProcessorContext context) {
    +        final MetricName dropTotalMetric = new MetricName(
    +            "dropped-records-total",
    +            "stream-task-metrics",
    +            "The total number of dropped records",
    +            mkMap(
    +                mkEntry("thread-id", Thread.currentThread().getName()),
    +                mkEntry("task-id", "0_0")
    +            )
    +        );
    +
    +        return context.metrics().metrics().get(dropTotalMetric).metricValue();
    +    }
    +
    +    private Object getDroppedRecordsRateMetric(final InternalProcessorContext context) {
    +        final MetricName dropRateMetric = new MetricName(
    +            "dropped-records-rate",
    +            "stream-task-metrics",
    +            "The average number of dropped records per second",
    +            mkMap(
    +                mkEntry("thread-id", Thread.currentThread().getName()),
    +                mkEntry("task-id", "0_0")
    +            )
    +        );
    +
    +        return context.metrics().metrics().get(dropRateMetric).metricValue();
    +    }
    +
     }
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/ReadOnlyStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/ReadOnlyStoreTest.java
    new file mode 100644
    index 0000000000..a786c82bf2
    --- /dev/null
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/ReadOnlyStoreTest.java
    @@ -0,0 +1,132 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements. See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License. You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.kafka.streams.processor;
    +
    +import org.apache.kafka.common.serialization.IntegerDeserializer;
    +import org.apache.kafka.common.serialization.IntegerSerializer;
    +import org.apache.kafka.common.serialization.Serdes;
    +import org.apache.kafka.common.serialization.StringDeserializer;
    +import org.apache.kafka.common.serialization.StringSerializer;
    +import org.apache.kafka.streams.KeyValue;
    +import org.apache.kafka.streams.TestInputTopic;
    +import org.apache.kafka.streams.TestOutputTopic;
    +import org.apache.kafka.streams.Topology;
    +import org.apache.kafka.streams.TopologyTestDriver;
    +import org.apache.kafka.streams.processor.api.Processor;
    +import org.apache.kafka.streams.processor.api.ProcessorContext;
    +import org.apache.kafka.streams.processor.api.Record;
    +import org.apache.kafka.streams.state.KeyValueIterator;
    +import org.apache.kafka.streams.state.KeyValueStore;
    +import org.apache.kafka.streams.state.Stores;
    +import org.junit.Test;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +
    +import static org.hamcrest.MatcherAssert.assertThat;
    +import static org.hamcrest.Matchers.equalTo;
    +
    +public class ReadOnlyStoreTest {
    +
    +    @Test
    +    public void shouldConnectProcessorAndWriteDataToReadOnlyStore() {
    +        final Topology topology = new Topology();
    +        topology.addReadOnlyStateStore(
    +            Stores.keyValueStoreBuilder(
    +                Stores.inMemoryKeyValueStore("readOnlyStore"),
    +                new Serdes.IntegerSerde(),
    +                new Serdes.StringSerde()
    +            ),
    +            "readOnlySource",
    +            new IntegerDeserializer(),
    +            new StringDeserializer(),
    +            "storeTopic",
    +            "readOnlyProcessor",
    +            () -> new Processor() {
    +                KeyValueStore store;
    +
    +                @Override
    +                public void init(final ProcessorContext context) {
    +                    store = context.getStateStore("readOnlyStore");
    +                }
    +                @Override
    +                public void process(final Record record) {
    +                    store.put(record.key(), record.value());
    +                }
    +            }
    +        );
    +
    +        topology.addSource("source", new IntegerDeserializer(), new StringDeserializer(), "inputTopic");
    +        topology.addProcessor(
    +            "processor",
    +            () -> new Processor() {
    +                ProcessorContext context;
    +                KeyValueStore store;
    +
    +                @Override
    +                public void init(final ProcessorContext context) {
    +                    this.context = context;
    +                    store = context.getStateStore("readOnlyStore");
    +                }
    +
    +                @Override
    +                public void process(final Record record) {
    +                    context.forward(record.withValue(
    +                        record.value() + " -- " + store.get(record.key())
    +                    ));
    +                }
    +            },
    +            "source"
    +        );
    +        topology.connectProcessorAndStateStores("processor", "readOnlyStore");
    +        topology.addSink("sink", "outputTopic", new IntegerSerializer(), new StringSerializer(), "processor");
    +
    +        try (final TopologyTestDriver driver = new TopologyTestDriver(topology)) {
    +            final TestInputTopic readOnlyStoreTopic =
    +                driver.createInputTopic("storeTopic", new IntegerSerializer(), new StringSerializer());
    +            final TestInputTopic input =
    +                driver.createInputTopic("inputTopic", new IntegerSerializer(), new StringSerializer());
    +            final TestOutputTopic output =
    +                driver.createOutputTopic("outputTopic", new IntegerDeserializer(), new StringDeserializer());
    +
    +            readOnlyStoreTopic.pipeInput(1, "foo");
    +            readOnlyStoreTopic.pipeInput(2, "bar");
    +
    +            input.pipeInput(1, "bar");
    +            input.pipeInput(2, "foo");
    +
    +            final KeyValueStore store = driver.getKeyValueStore("readOnlyStore");
    +
    +            try (final KeyValueIterator it = store.all()) {
    +                final List> storeContent = new LinkedList<>();
    +                it.forEachRemaining(storeContent::add);
    +
    +                final List> expectedResult = new LinkedList<>();
    +                expectedResult.add(KeyValue.pair(1, "foo"));
    +                expectedResult.add(KeyValue.pair(2, "bar"));
    +
    +                assertThat(storeContent, equalTo(expectedResult));
    +            }
    +
    +            final List> expectedResult = new LinkedList<>();
    +            expectedResult.add(KeyValue.pair(1, "bar -- foo"));
    +            expectedResult.add(KeyValue.pair(2, "foo -- bar"));
    +
    +            assertThat(output.readKeyValuesToList(), equalTo(expectedResult));
    +        }
    +    }
    +}
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdaterTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdaterTest.java
    index 4d4728edab..4f6a659aca 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdaterTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdaterTest.java
    @@ -61,6 +61,7 @@
     import static org.hamcrest.Matchers.not;
     import static org.junit.jupiter.api.Assertions.assertEquals;
     import static org.junit.jupiter.api.Assertions.assertFalse;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     import static org.junit.jupiter.api.Assertions.assertThrows;
     import static org.junit.jupiter.api.Assertions.assertTrue;
     import static org.mockito.ArgumentMatchers.anyBoolean;
    @@ -1676,7 +1677,7 @@ private void verifyGetTasks(final Set expectedActiveTasks,
             final Set tasks = stateUpdater.getTasks();
     
             assertEquals(expectedActiveTasks.size() + expectedStandbyTasks.size(), tasks.size());
    -        tasks.forEach(task -> assertTrue(task instanceof ReadOnlyTask));
    +        tasks.forEach(task -> assertInstanceOf(ReadOnlyTask.class, task));
             final Set actualTaskIds = tasks.stream().map(Task::id).collect(Collectors.toSet());
             final Set expectedTasks = new HashSet<>(expectedActiveTasks);
             expectedTasks.addAll(expectedStandbyTasks);
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java
    index 31be9dc2a4..af5dc68103 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java
    @@ -25,6 +25,7 @@
     import org.apache.kafka.common.serialization.LongSerializer;
     import org.apache.kafka.common.serialization.StringDeserializer;
     import org.apache.kafka.common.utils.LogContext;
    +import org.apache.kafka.common.utils.MockTime;
     import org.apache.kafka.common.utils.Utils;
     import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler;
     import org.apache.kafka.streams.errors.LogAndFailExceptionHandler;
    @@ -46,8 +47,6 @@
     
     import static java.util.Arrays.asList;
     import static org.apache.kafka.streams.processor.internals.testutil.ConsumerRecordUtil.record;
    -import static org.hamcrest.CoreMatchers.equalTo;
    -import static org.hamcrest.MatcherAssert.assertThat;
     import static org.junit.Assert.assertEquals;
     import static org.junit.Assert.assertFalse;
     import static org.junit.Assert.assertTrue;
    @@ -71,8 +70,12 @@ public class GlobalStateTaskTest {
         private final MockProcessorNode processorTwo = new MockProcessorNode<>();
     
         private final Map offsets = new HashMap<>();
    -    private File testDirectory = TestUtils.tempDirectory("global-store");
    +    private final File testDirectory = TestUtils.tempDirectory("global-store");
         private final NoOpProcessorContext context = new NoOpProcessorContext();
    +    private final MockTime time = new MockTime();
    +    private final long flushInterval = 1000L;
    +    private final long currentOffsetT1 = 50;
    +    private final long currentOffsetT2 = 100;
     
         private ProcessorTopology topology;
         private GlobalStateManagerStub stateMgr;
    @@ -101,7 +104,9 @@ public void before() {
                 topology,
                 context,
                 stateMgr,
    -            new LogAndFailExceptionHandler()
    +            new LogAndFailExceptionHandler(),
    +            time,
    +            flushInterval
             );
         }
     
    @@ -188,7 +193,9 @@ public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHan
                 topology,
                 context,
                 stateMgr,
    -            new LogAndContinueExceptionHandler()
    +            new LogAndContinueExceptionHandler(),
    +            time,
    +            flushInterval
             );
             final byte[] key = new LongSerializer().serialize(topic2, 1L);
             final byte[] recordValue = new IntegerSerializer().serialize(topic2, 10);
    @@ -203,7 +210,9 @@ public void shouldNotThrowStreamsExceptionWhenValueDeserializationFails() {
                 topology,
                 context,
                 stateMgr,
    -            new LogAndContinueExceptionHandler()
    +            new LogAndContinueExceptionHandler(),
    +            time,
    +            flushInterval
             );
             final byte[] key = new IntegerSerializer().serialize(topic2, 1);
             final byte[] recordValue = new LongSerializer().serialize(topic2, 10L);
    @@ -217,10 +226,13 @@ public void shouldFlushStateManagerWithOffsets() {
             final Map expectedOffsets = new HashMap<>();
             expectedOffsets.put(t1, 52L);
             expectedOffsets.put(t2, 100L);
    +
             globalStateTask.initialize();
    -        globalStateTask.update(record(topic1, 1, 51, "foo".getBytes(), "foo".getBytes()));
    +        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 1, "foo".getBytes(), "foo".getBytes()));
             globalStateTask.flushState();
    +
             assertEquals(expectedOffsets, stateMgr.changelogOffsets());
    +        assertTrue(stateMgr.flushed);
         }
     
         @Test
    @@ -228,12 +240,93 @@ public void shouldCheckpointOffsetsWhenStateIsFlushed() {
             final Map expectedOffsets = new HashMap<>();
             expectedOffsets.put(t1, 102L);
             expectedOffsets.put(t2, 100L);
    +
             globalStateTask.initialize();
    -        globalStateTask.update(record(topic1, 1, 101, "foo".getBytes(), "foo".getBytes()));
    +        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 51L, "foo".getBytes(), "foo".getBytes()));
             globalStateTask.flushState();
    -        assertThat(stateMgr.changelogOffsets(), equalTo(expectedOffsets));
    +
    +        assertEquals(expectedOffsets, stateMgr.changelogOffsets());
    +        assertTrue(stateMgr.checkpointWritten);
    +    }
    +
    +    @Test
    +    public void shouldNotCheckpointIfNotReceivedEnoughRecords() {
    +        globalStateTask.initialize();
    +        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 9000L, "foo".getBytes(), "foo".getBytes()));
    +        time.sleep(flushInterval); // flush interval elapsed
    +        globalStateTask.maybeCheckpoint();
    +
    +        assertEquals(offsets, stateMgr.changelogOffsets());
    +        assertFalse(stateMgr.flushed);
    +        assertFalse(stateMgr.checkpointWritten);
    +    }
    +
    +    @Test
    +    public void shouldNotCheckpointWhenFlushIntervalHasNotLapsed() {
    +        globalStateTask.initialize();
    +
    +        // offset delta exceeded
    +        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 10000L, "foo".getBytes(), "foo".getBytes()));
    +
    +        time.sleep(flushInterval / 2);
    +        globalStateTask.maybeCheckpoint();
    +
    +        assertEquals(offsets, stateMgr.changelogOffsets());
    +        assertFalse(stateMgr.flushed);
    +        assertFalse(stateMgr.checkpointWritten);
    +    }
    +
    +    @Test
    +    public void shouldCheckpointIfReceivedEnoughRecordsAndFlushIntervalHasElapsed() {
    +        final Map expectedOffsets = new HashMap<>();
    +        expectedOffsets.put(t1, 10051L); // topic1 advanced with 10001 records
    +        expectedOffsets.put(t2, 100L);
    +
    +        globalStateTask.initialize();
    +
    +        time.sleep(flushInterval); // flush interval elapsed
    +
    +        // 10000 records received since last flush => do not flush
    +        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 9999L, "foo".getBytes(), "foo".getBytes()));
    +        globalStateTask.maybeCheckpoint();
    +
    +        assertEquals(offsets, stateMgr.changelogOffsets());
    +        assertFalse(stateMgr.flushed);
    +        assertFalse(stateMgr.checkpointWritten);
    +
    +        // 1 more record received => triggers the flush
    +        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 10000L, "foo".getBytes(), "foo".getBytes()));
    +        globalStateTask.maybeCheckpoint();
    +
    +        assertEquals(expectedOffsets, stateMgr.changelogOffsets());
    +        assertTrue(stateMgr.flushed);
    +        assertTrue(stateMgr.checkpointWritten);
         }
     
    +    @Test
    +    public void shouldCheckpointIfReceivedEnoughRecordsFromMultipleTopicsAndFlushIntervalElapsed() {
    +        final byte[] integerBytes = new IntegerSerializer().serialize(topic2, 1);
    +
    +        final Map expectedOffsets = new HashMap<>();
    +        expectedOffsets.put(t1, 9050L); // topic1 advanced with 9000 records
    +        expectedOffsets.put(t2, 1101L); // topic2 advanced with 1001 records
    +
    +        globalStateTask.initialize();
    +
    +        time.sleep(flushInterval);
    +
    +        // received 9000 records in topic1
    +        globalStateTask.update(record(topic1, 1, currentOffsetT1 + 8999L, "foo".getBytes(), "foo".getBytes()));
    +        // received 1001 records in topic2
    +        globalStateTask.update(record(topic2, 1, currentOffsetT2 + 1000L, integerBytes, integerBytes));
    +        globalStateTask.maybeCheckpoint();
    +
    +        assertEquals(expectedOffsets, stateMgr.changelogOffsets());
    +        assertTrue(stateMgr.flushed);
    +        assertTrue(stateMgr.checkpointWritten);
    +    }
    +
    +
         @Test
         public void shouldWipeGlobalStateDirectory() throws Exception {
             assertTrue(stateMgr.baseDir().exists());
    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 a63e7e884a..63f00b8cb7 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
    @@ -78,6 +78,7 @@
     import static org.junit.Assert.assertThrows;
     import static org.junit.Assert.assertTrue;
     import static org.junit.Assert.fail;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     
     public class InternalTopologyBuilderTest {
     
    @@ -891,14 +892,14 @@ public void shouldAddInternalTopicConfigForWindowStores() {
             assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE, properties1.get(TopicConfig.CLEANUP_POLICY_CONFIG));
             assertEquals("40000", properties1.get(TopicConfig.RETENTION_MS_CONFIG));
             assertEquals("appId-store1-changelog", topicConfig1.name());
    -        assertTrue(topicConfig1 instanceof WindowedChangelogTopicConfig);
    +        assertInstanceOf(WindowedChangelogTopicConfig.class, topicConfig1);
             final InternalTopicConfig topicConfig2 = topicsInfo.stateChangelogTopics.get("appId-store2-changelog");
             final Map properties2 = topicConfig2.getProperties(Collections.emptyMap(), 10000);
             assertEquals(3, properties2.size());
             assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE, properties2.get(TopicConfig.CLEANUP_POLICY_CONFIG));
             assertEquals("40000", properties2.get(TopicConfig.RETENTION_MS_CONFIG));
             assertEquals("appId-store2-changelog", topicConfig2.name());
    -        assertTrue(topicConfig2 instanceof WindowedChangelogTopicConfig);
    +        assertInstanceOf(WindowedChangelogTopicConfig.class, topicConfig2);
         }
     
         @Test
    @@ -923,7 +924,7 @@ public void shouldAddInternalTopicConfigForVersionedStores() {
             assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, properties.get(TopicConfig.CLEANUP_POLICY_CONFIG));
             assertEquals(Long.toString(60_000L + 24 * 60 * 60 * 1000L), properties.get(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG));
             assertEquals("appId-vstore-changelog", topicConfig.name());
    -        assertTrue(topicConfig instanceof VersionedChangelogTopicConfig);
    +        assertInstanceOf(VersionedChangelogTopicConfig.class, topicConfig);
         }
     
         @Test
    @@ -940,7 +941,7 @@ public void shouldAddInternalTopicConfigForNonWindowNonVersionedStores() {
             assertEquals(2, properties.size());
             assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, properties.get(TopicConfig.CLEANUP_POLICY_CONFIG));
             assertEquals("appId-testStore-changelog", topicConfig.name());
    -        assertTrue(topicConfig instanceof UnwindowedUnversionedChangelogTopicConfig);
    +        assertInstanceOf(UnwindowedUnversionedChangelogTopicConfig.class, topicConfig);
         }
     
         @Test
    @@ -956,7 +957,7 @@ public void shouldAddInternalTopicConfigForRepartitionTopics() {
             assertEquals(String.valueOf(-1), properties.get(TopicConfig.RETENTION_MS_CONFIG));
             assertEquals(TopicConfig.CLEANUP_POLICY_DELETE, properties.get(TopicConfig.CLEANUP_POLICY_CONFIG));
             assertEquals("appId-foo", topicConfig.name());
    -        assertTrue(topicConfig instanceof RepartitionTopicConfig);
    +        assertInstanceOf(RepartitionTopicConfig.class, topicConfig);
         }
     
         @Test
    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 fef0fc055b..8de868f0e8 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
    @@ -81,6 +81,7 @@
     import static org.junit.Assert.assertThrows;
     import static org.junit.Assert.assertTrue;
     import static org.junit.Assert.fail;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     import static org.mockito.Mockito.doThrow;
     import static org.mockito.Mockito.mock;
     import static org.mockito.Mockito.times;
    @@ -996,7 +997,7 @@ public void shouldThrowIllegalStateIfInitializingOffsetsForCorruptedTasks() {
                 stateMgr.markChangelogAsCorrupted(mkSet(persistentStorePartition));
     
                 final ProcessorStateException thrown = assertThrows(ProcessorStateException.class, () -> stateMgr.initializeStoreOffsetsFromCheckpoint(true));
    -            assertTrue(thrown.getCause() instanceof IllegalStateException);
    +            assertInstanceOf(IllegalStateException.class, thrown.getCause());
             } finally {
                 stateMgr.close();
             }
    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 1f98eb456d..5e57939483 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
    @@ -21,7 +21,6 @@
     import org.apache.kafka.clients.consumer.OffsetResetStrategy;
     import org.apache.kafka.common.TopicPartition;
     import org.apache.kafka.common.utils.LogContext;
    -import org.apache.kafka.common.utils.MockTime;
     import org.apache.kafka.common.utils.Utils;
     import org.junit.Before;
     import org.junit.Test;
    @@ -32,16 +31,13 @@
     import java.util.Map;
     
     import static org.junit.Assert.assertEquals;
    -import static org.junit.Assert.assertFalse;
     import static org.junit.Assert.assertTrue;
     
     
     public class StateConsumerTest {
     
    -    private static final long FLUSH_INTERVAL = 1000L;
         private final TopicPartition topicOne = new TopicPartition("topic-one", 1);
         private final TopicPartition topicTwo = new TopicPartition("topic-two", 1);
    -    private final MockTime time = new MockTime();
         private final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
         private final Map partitionOffsets = new HashMap<>();
         private final LogContext logContext = new LogContext("test ");
    @@ -53,7 +49,7 @@ public void setUp() {
             partitionOffsets.put(topicOne, 20L);
             partitionOffsets.put(topicTwo, 30L);
             stateMaintainer = new TaskStub(partitionOffsets);
    -        stateConsumer = new GlobalStreamThread.StateConsumer(logContext, consumer, stateMaintainer, time, Duration.ofMillis(10L), FLUSH_INTERVAL);
    +        stateConsumer = new GlobalStreamThread.StateConsumer(logContext, consumer, stateMaintainer, Duration.ofMillis(10L));
         }
     
         @Test
    @@ -76,6 +72,7 @@ public void shouldUpdateStateWithReceivedRecordsForPartition() {
             consumer.addRecord(new ConsumerRecord<>("topic-one", 1, 21L, new byte[0], new byte[0]));
             stateConsumer.pollAndUpdate();
             assertEquals(2, stateMaintainer.updatedPartitions.get(topicOne).intValue());
    +        assertTrue(stateMaintainer.flushed);
         }
     
         @Test
    @@ -87,27 +84,9 @@ public void shouldUpdateStateWithReceivedRecordsForAllTopicPartition() {
             stateConsumer.pollAndUpdate();
             assertEquals(1, stateMaintainer.updatedPartitions.get(topicOne).intValue());
             assertEquals(2, stateMaintainer.updatedPartitions.get(topicTwo).intValue());
    -    }
    -
    -    @Test
    -    public void shouldFlushStoreWhenFlushIntervalHasLapsed() {
    -        stateConsumer.initialize();
    -        consumer.addRecord(new ConsumerRecord<>("topic-one", 1, 20L, new byte[0], new byte[0]));
    -        time.sleep(FLUSH_INTERVAL);
    -
    -        stateConsumer.pollAndUpdate();
             assertTrue(stateMaintainer.flushed);
         }
     
    -    @Test
    -    public void shouldNotFlushOffsetsWhenFlushIntervalHasNotLapsed() {
    -        stateConsumer.initialize();
    -        consumer.addRecord(new ConsumerRecord<>("topic-one", 1, 20L, new byte[0], new byte[0]));
    -        time.sleep(FLUSH_INTERVAL / 2);
    -        stateConsumer.pollAndUpdate();
    -        assertFalse(stateMaintainer.flushed);
    -    }
    -
         @Test
         public void shouldCloseConsumer() throws IOException {
             stateConsumer.close(false);
    @@ -161,6 +140,10 @@ public void update(final ConsumerRecord record) {
                 updatedPartitions.put(tp, updatedPartitions.get(tp) + 1);
             }
     
    +        @Override
    +        public void maybeCheckpoint() {
    +            flushState();
    +        }
         }
     
    -}
    \ No newline at end of file
    +}
    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 f0feebb73f..44644ba157 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
    @@ -29,7 +29,6 @@
     import org.apache.kafka.clients.producer.MockProducer;
     import org.apache.kafka.clients.producer.Producer;
     import org.apache.kafka.common.Cluster;
    -import org.apache.kafka.common.KafkaException;
     import org.apache.kafka.common.KafkaFuture;
     import org.apache.kafka.common.Metric;
     import org.apache.kafka.common.MetricName;
    @@ -161,7 +160,6 @@
     import static org.junit.Assert.assertSame;
     import static org.junit.Assert.assertThrows;
     import static org.junit.Assert.assertTrue;
    -import static org.junit.Assert.fail;
     import static org.mockito.ArgumentMatchers.any;
     import static org.mockito.ArgumentMatchers.anyLong;
     import static org.mockito.ArgumentMatchers.anyString;
    @@ -1634,18 +1632,16 @@ public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerWasFencedWhilePr
             mockTime.sleep(config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG) + 1L);
             consumer.addRecord(new ConsumerRecord<>(topic1, 1, 1, new byte[0], new byte[0]));
     
    -        try {
    -            if (processingThreadsEnabled) {
    -                runUntilTimeoutOrException(this::runOnce);
    -            } else {
    -                runOnce();
    -            }
    -            fail("Should have thrown TaskMigratedException");
    -        } catch (final KafkaException expected) {
    -            assertTrue(String.format("Expected TaskMigratedException but got %s", expected), expected instanceof TaskMigratedException);
    -            assertTrue("StreamsThread removed the fenced zombie task already, should wait for rebalance to close all zombies together.",
    -                thread.readOnlyActiveTasks().stream().anyMatch(task -> task.id().equals(task1)));
    -        }
    +        assertThrows(TaskMigratedException.class,
    +            () -> {
    +                if (processingThreadsEnabled) {
    +                    runUntilTimeoutOrException(this::runOnce);
    +                } else {
    +                    runOnce();
    +                }
    +            });
    +        assertTrue("StreamsThread removed the fenced zombie task already, should wait for rebalance to close all zombies together.",
    +            thread.readOnlyActiveTasks().stream().anyMatch(task -> task.id().equals(task1)));
     
             assertThat(producer.commitCount(), equalTo(1L));
         }
    @@ -1847,18 +1843,16 @@ public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerGotFencedInCommi
             producer.commitTransactionException = new ProducerFencedException("Producer is fenced");
             mockTime.sleep(config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG) + 1L);
             consumer.addRecord(new ConsumerRecord<>(topic1, 1, 1, new byte[0], new byte[0]));
    -        try {
    -            if (processingThreadsEnabled) {
    -                runUntilTimeoutOrException(this::runOnce);
    -            } else {
    -                runOnce();
    -            }
    -            fail("Should have thrown TaskMigratedException");
    -        } catch (final KafkaException expected) {
    -            assertTrue(expected instanceof TaskMigratedException);
    -            assertTrue("StreamsThread removed the fenced zombie task already, should wait for rebalance to close all zombies together.",
    -                thread.readOnlyActiveTasks().stream().anyMatch(task -> task.id().equals(task1)));
    -        }
    +        assertThrows(TaskMigratedException.class,
    +            () -> {
    +                if (processingThreadsEnabled) {
    +                    runUntilTimeoutOrException(this::runOnce);
    +                } else {
    +                    runOnce();
    +                }
    +            });
    +        assertTrue("StreamsThread removed the fenced zombie task already, should wait for rebalance to close all zombies together.",
    +            thread.readOnlyActiveTasks().stream().anyMatch(task -> task.id().equals(task1)));
     
             assertThat(producer.commitCount(), equalTo(0L));
     
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsAssignmentScaleTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsAssignmentScaleTest.java
    index 834b624219..e14e033363 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsAssignmentScaleTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsAssignmentScaleTest.java
    @@ -68,73 +68,73 @@
     @Category({IntegrationTest.class})
     @RunWith(MockitoJUnitRunner.StrictStubs.class)
     public class StreamsAssignmentScaleTest {
    -    final static long MAX_ASSIGNMENT_DURATION = 60 * 1000L; //each individual assignment should complete within 20s
    +    final static long MAX_ASSIGNMENT_DURATION = 120 * 1000L; // we should stay below `max.poll.interval.ms`
         final static String APPLICATION_ID = "streams-assignment-scale-test";
     
         private final Logger log = LoggerFactory.getLogger(StreamsAssignmentScaleTest.class);
     
         /* HighAvailabilityTaskAssignor tests */
     
    -    @Test(timeout = 120 * 1000)
    +    @Test(timeout = 300 * 1000)
         public void testHighAvailabilityTaskAssignorLargePartitionCount() {
             completeLargeAssignment(6_000, 2, 1, 1, HighAvailabilityTaskAssignor.class);
         }
     
    -    @Test(timeout = 120 * 1000)
    +    @Test(timeout = 300 * 1000)
         public void testHighAvailabilityTaskAssignorLargeNumConsumers() {
             completeLargeAssignment(1_000, 1_000, 1, 1, HighAvailabilityTaskAssignor.class);
         }
     
    -    @Test(timeout = 120 * 1000)
    +    @Test(timeout = 300 * 1000)
         public void testHighAvailabilityTaskAssignorManyStandbys() {
             completeLargeAssignment(1_000, 100, 1, 50, HighAvailabilityTaskAssignor.class);
         }
     
    -    @Test(timeout = 120 * 1000)
    +    @Test(timeout = 300 * 1000)
         public void testHighAvailabilityTaskAssignorManyThreadsPerClient() {
             completeLargeAssignment(1_000, 10, 1000, 1, HighAvailabilityTaskAssignor.class);
         }
     
         /* StickyTaskAssignor tests */
     
    -    @Test(timeout = 120 * 1000)
    +    @Test(timeout = 300 * 1000)
         public void testStickyTaskAssignorLargePartitionCount() {
             completeLargeAssignment(2_000, 2, 1, 1, StickyTaskAssignor.class);
         }
     
    -    @Test(timeout = 120 * 1000)
    +    @Test(timeout = 300 * 1000)
         public void testStickyTaskAssignorLargeNumConsumers() {
             completeLargeAssignment(1_000, 1_000, 1, 1, StickyTaskAssignor.class);
         }
     
    -    @Test(timeout = 120 * 1000)
    +    @Test(timeout = 300 * 1000)
         public void testStickyTaskAssignorManyStandbys() {
             completeLargeAssignment(1_000, 100, 1, 20, StickyTaskAssignor.class);
         }
     
    -    @Test(timeout = 120 * 1000)
    +    @Test(timeout = 300 * 1000)
         public void testStickyTaskAssignorManyThreadsPerClient() {
             completeLargeAssignment(1_000, 10, 1000, 1, StickyTaskAssignor.class);
         }
     
         /* FallbackPriorTaskAssignor tests */
     
    -    @Test(timeout = 120 * 1000)
    +    @Test(timeout = 300 * 1000)
         public void testFallbackPriorTaskAssignorLargePartitionCount() {
             completeLargeAssignment(2_000, 2, 1, 1, FallbackPriorTaskAssignor.class);
         }
     
    -    @Test(timeout = 120 * 1000)
    +    @Test(timeout = 300 * 1000)
         public void testFallbackPriorTaskAssignorLargeNumConsumers() {
             completeLargeAssignment(1_000, 1_000, 1, 1, FallbackPriorTaskAssignor.class);
         }
     
    -    @Test(timeout = 120 * 1000)
    +    @Test(timeout = 300 * 1000)
         public void testFallbackPriorTaskAssignorManyStandbys() {
             completeLargeAssignment(1_000, 100, 1, 20, FallbackPriorTaskAssignor.class);
         }
     
    -    @Test(timeout = 120 * 1000)
    +    @Test(timeout = 300 * 1000)
         public void testFallbackPriorTaskAssignorManyThreadsPerClient() {
             completeLargeAssignment(1_000, 10, 1000, 1, FallbackPriorTaskAssignor.class);
         }
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java
    index ba1c91e7f7..64ad0d1caf 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java
    @@ -58,10 +58,6 @@
     import java.time.Duration;
     import java.util.ArrayList;
     
    -import org.easymock.EasyMock;
    -import org.easymock.EasyMockRunner;
    -import org.easymock.Mock;
    -import org.easymock.MockType;
     import org.hamcrest.Matchers;
     import org.junit.Before;
     import org.junit.Rule;
    @@ -70,10 +66,10 @@
     import org.junit.runner.RunWith;
     import org.mockito.Answers;
     import org.mockito.InOrder;
    -import org.mockito.Mockito;
    +import org.mockito.Mock;
     import org.mockito.junit.MockitoJUnit;
    +import org.mockito.junit.MockitoJUnitRunner;
     import org.mockito.junit.MockitoRule;
    -import org.mockito.junit.jupiter.MockitoSettings;
     import org.mockito.quality.Strictness;
     
     import java.io.File;
    @@ -108,12 +104,6 @@
     import static org.apache.kafka.test.StreamsTestUtils.TaskBuilder.standbyTask;
     import static org.apache.kafka.test.StreamsTestUtils.TaskBuilder.statefulTask;
     import static org.apache.kafka.test.StreamsTestUtils.TaskBuilder.statelessTask;
    -import static org.easymock.EasyMock.eq;
    -import static org.easymock.EasyMock.expect;
    -import static org.easymock.EasyMock.expectLastCall;
    -import static org.easymock.EasyMock.replay;
    -import static org.easymock.EasyMock.reset;
    -import static org.easymock.EasyMock.verify;
     import static org.hamcrest.CoreMatchers.hasItem;
     import static org.hamcrest.MatcherAssert.assertThat;
     import static org.hamcrest.Matchers.empty;
    @@ -127,19 +117,25 @@
     import static org.junit.Assert.assertTrue;
     import static org.junit.Assert.fail;
     import static org.mockito.ArgumentMatchers.any;
    +import static org.mockito.ArgumentMatchers.anyBoolean;
     import static org.mockito.ArgumentMatchers.anyLong;
     import static org.mockito.ArgumentMatchers.anyMap;
    +import static org.mockito.ArgumentMatchers.anyString;
     import static org.mockito.ArgumentMatchers.argThat;
    +import static org.mockito.ArgumentMatchers.eq;
     import static org.mockito.Mockito.doNothing;
     import static org.mockito.Mockito.doThrow;
     import static org.mockito.Mockito.inOrder;
    +import static org.mockito.Mockito.lenient;
     import static org.mockito.Mockito.never;
     import static org.mockito.Mockito.times;
    +import static org.mockito.Mockito.verify;
    +import static org.mockito.Mockito.verifyNoInteractions;
    +import static org.mockito.Mockito.verifyNoMoreInteractions;
     import static org.mockito.Mockito.when;
     import static org.mockito.Mockito.mock;
     
    -@MockitoSettings(strictness = Strictness.STRICT_STUBS)
    -@RunWith(EasyMockRunner.class)
    +@RunWith(MockitoJUnitRunner.StrictStubs.class)
     public class TaskManagerTest {
     
         private final String topic1 = "topic1";
    @@ -185,31 +181,30 @@ public class TaskManagerTest {
         private final TaskId taskId10 = new TaskId(1, 0);
         private final TopicPartition t2p0 = new TopicPartition(topic2, 0);
         private final Set taskId10Partitions = mkSet(t2p0);
    +    private final Set assignment = singleton(new TopicPartition("assignment", 0));
     
         final java.util.function.Consumer> noOpResetter = partitions -> { };
     
    -    @org.mockito.Mock
    +    @Mock
         private InternalTopologyBuilder topologyBuilder;
    -    @Mock(type = MockType.DEFAULT)
    +    @Mock
         private StateDirectory stateDirectory;
    -    @org.mockito.Mock
    +    @Mock
         private ChangelogReader changeLogReader;
    -    @Mock(type = MockType.STRICT)
    +    @Mock
         private Consumer consumer;
    -    @org.mockito.Mock
    -    private Consumer mockitoConsumer;
    -    @org.mockito.Mock
    +    @Mock
         private ActiveTaskCreator activeTaskCreator;
    -    @org.mockito.Mock
    +    @Mock
         private StandbyTaskCreator standbyTaskCreator;
    -    @org.mockito.Mock
    +    @Mock
         private Admin adminClient;
    -    @org.mockito.Mock
    +    @Mock
         private ProcessorStateManager stateManager;
    -    @org.mockito.Mock(answer = Answers.RETURNS_DEEP_STUBS)
    +    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
         private ProcessorStateManager.StateStoreMetadata stateStore;
    -    final StateUpdater stateUpdater = Mockito.mock(StateUpdater.class);
    -    final DefaultTaskManager schedulingTaskManager = Mockito.mock(DefaultTaskManager.class);
    +    final StateUpdater stateUpdater = mock(StateUpdater.class);
    +    final DefaultTaskManager schedulingTaskManager = mock(DefaultTaskManager.class);
     
         private TaskManager taskManager;
         private TopologyMetadata topologyMetadata;
    @@ -269,7 +264,7 @@ public void shouldClassifyExistingTasksWithoutStateUpdater() {
     
             taskManager.handleAssignment(activeTasks, standbyTasks);
     
    -        Mockito.verifyNoInteractions(stateUpdater);
    +        verifyNoInteractions(stateUpdater);
         }
     
         @Test
    @@ -278,7 +273,7 @@ public void shouldNotUpdateExistingStandbyTaskIfStandbyIsReassignedWithSameInput
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId03Partitions).build();
             updateExistingStandbyTaskIfStandbyIsReassignedWithoutStateUpdater(standbyTask, taskId03Partitions);
    -        Mockito.verify(standbyTask, never()).updateInputPartitions(Mockito.eq(taskId03Partitions), Mockito.any());
    +        verify(standbyTask, never()).updateInputPartitions(eq(taskId03Partitions), any());
         }
     
         @Test
    @@ -287,12 +282,12 @@ public void shouldUpdateExistingStandbyTaskIfStandbyIsReassignedWithDifferentInp
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId03Partitions).build();
             updateExistingStandbyTaskIfStandbyIsReassignedWithoutStateUpdater(standbyTask, taskId04Partitions);
    -        Mockito.verify(standbyTask).updateInputPartitions(Mockito.eq(taskId04Partitions), Mockito.any());
    +        verify(standbyTask).updateInputPartitions(eq(taskId04Partitions), any());
         }
     
         private void updateExistingStandbyTaskIfStandbyIsReassignedWithoutStateUpdater(final Task standbyTask,
                                                                                        final Set newInputPartition) {
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             when(tasks.allTasks()).thenReturn(mkSet(standbyTask));
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, false);
     
    @@ -301,7 +296,7 @@ private void updateExistingStandbyTaskIfStandbyIsReassignedWithoutStateUpdater(f
                 mkMap(mkEntry(standbyTask.id(), newInputPartition))
             );
     
    -        Mockito.verify(standbyTask).resume();
    +        verify(standbyTask).resume();
         }
     
         @Test
    @@ -309,9 +304,8 @@ public void shouldLockAllTasksOnCorruptionWithProcessingThreads() {
             final StreamTask activeTask1 = statefulTask(taskId00, taskId00ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId00Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true, true);
    -        taskManager.setMainConsumer(mockitoConsumer);
             when(tasks.activeTaskIds()).thenReturn(mkSet(taskId00, taskId01));
             when(tasks.task(taskId00)).thenReturn(activeTask1);
             final KafkaFuture mockFuture = KafkaFuture.completedFuture(null);
    @@ -319,9 +313,9 @@ public void shouldLockAllTasksOnCorruptionWithProcessingThreads() {
     
             taskManager.handleCorruption(mkSet(taskId00));
     
    -        Mockito.verify(mockitoConsumer).assignment();
    -        Mockito.verify(schedulingTaskManager).lockTasks(mkSet(taskId00, taskId01));
    -        Mockito.verify(schedulingTaskManager).unlockTasks(mkSet(taskId00, taskId01));
    +        verify(consumer).assignment();
    +        verify(schedulingTaskManager).lockTasks(mkSet(taskId00, taskId01));
    +        verify(schedulingTaskManager).unlockTasks(mkSet(taskId00, taskId01));
         }
     
         @Test
    @@ -332,20 +326,20 @@ public void shouldLockCommitableTasksOnCorruptionWithProcessingThreads() {
             final StreamTask activeTask2 = statefulTask(taskId01, taskId01ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId01Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true, true);
             final KafkaFuture mockFuture = KafkaFuture.completedFuture(null);
             when(schedulingTaskManager.lockTasks(any())).thenReturn(mockFuture);
     
             taskManager.commit(mkSet(activeTask1, activeTask2));
     
    -        Mockito.verify(schedulingTaskManager).lockTasks(mkSet(taskId00, taskId01));
    -        Mockito.verify(schedulingTaskManager).unlockTasks(mkSet(taskId00, taskId01));
    +        verify(schedulingTaskManager).lockTasks(mkSet(taskId00, taskId01));
    +        verify(schedulingTaskManager).unlockTasks(mkSet(taskId00, taskId01));
         }
     
         @Test
         public void shouldLockActiveOnHandleAssignmentWithProcessingThreads() {
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true, true);
             when(tasks.allTaskIds()).thenReturn(mkSet(taskId00, taskId01));
             final KafkaFuture mockFuture = KafkaFuture.completedFuture(null);
    @@ -356,8 +350,8 @@ public void shouldLockActiveOnHandleAssignmentWithProcessingThreads() {
                     mkMap(mkEntry(taskId01, taskId01Partitions))
             );
     
    -        Mockito.verify(schedulingTaskManager).lockTasks(mkSet(taskId00, taskId01));
    -        Mockito.verify(schedulingTaskManager).unlockTasks(mkSet(taskId00, taskId01));
    +        verify(schedulingTaskManager).lockTasks(mkSet(taskId00, taskId01));
    +        verify(schedulingTaskManager).unlockTasks(mkSet(taskId00, taskId01));
         }
     
         @Test
    @@ -368,7 +362,7 @@ public void shouldLockAffectedTasksOnHandleRevocation() {
             final StreamTask activeTask2 = statefulTask(taskId01, taskId01ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId01Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true, true);
             when(tasks.allTasks()).thenReturn(mkSet(activeTask1, activeTask2));
             final KafkaFuture mockFuture = KafkaFuture.completedFuture(null);
    @@ -376,8 +370,8 @@ public void shouldLockAffectedTasksOnHandleRevocation() {
     
             taskManager.handleRevocation(taskId01Partitions);
     
    -        Mockito.verify(schedulingTaskManager).lockTasks(mkSet(taskId00, taskId01));
    -        Mockito.verify(schedulingTaskManager).unlockTasks(mkSet(taskId00, taskId01));
    +        verify(schedulingTaskManager).lockTasks(mkSet(taskId00, taskId01));
    +        verify(schedulingTaskManager).unlockTasks(mkSet(taskId00, taskId01));
         }
     
         @Test
    @@ -388,7 +382,7 @@ public void shouldLockTasksOnClose() {
             final StreamTask activeTask2 = statefulTask(taskId01, taskId01ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId01Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true, true);
             when(tasks.allTasks()).thenReturn(mkSet(activeTask1, activeTask2));
             final KafkaFuture mockFuture = KafkaFuture.completedFuture(null);
    @@ -396,8 +390,8 @@ public void shouldLockTasksOnClose() {
     
             taskManager.closeAndCleanUpTasks(mkSet(activeTask1), mkSet(), false);
     
    -        Mockito.verify(schedulingTaskManager).lockTasks(mkSet(taskId00));
    -        Mockito.verify(schedulingTaskManager).unlockTasks(mkSet(taskId00));
    +        verify(schedulingTaskManager).lockTasks(mkSet(taskId00));
    +        verify(schedulingTaskManager).unlockTasks(mkSet(taskId00));
         }
     
         @Test
    @@ -408,14 +402,14 @@ public void shouldResumePollingForPartitionsWithAvailableSpaceForAllActiveTasks(
             final StreamTask activeTask2 = statefulTask(taskId01, taskId01ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId01Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(tasks.activeTasks()).thenReturn(mkSet(activeTask1, activeTask2));
     
             taskManager.resumePollingForPartitionsWithAvailableSpace();
     
    -        Mockito.verify(activeTask1).resumePollingForPartitionsWithAvailableSpace();
    -        Mockito.verify(activeTask2).resumePollingForPartitionsWithAvailableSpace();
    +        verify(activeTask1).resumePollingForPartitionsWithAvailableSpace();
    +        verify(activeTask2).resumePollingForPartitionsWithAvailableSpace();
         }
     
         @Test
    @@ -426,14 +420,14 @@ public void shouldUpdateLagForAllActiveTasks() {
             final StreamTask activeTask2 = statefulTask(taskId01, taskId01ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId01Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(tasks.activeTasks()).thenReturn(mkSet(activeTask1, activeTask2));
     
             taskManager.updateLags();
     
    -        Mockito.verify(activeTask1).updateLags();
    -        Mockito.verify(activeTask2).updateLags();
    +        verify(activeTask1).updateLags();
    +        verify(activeTask2).updateLags();
         }
     
         @Test
    @@ -441,7 +435,7 @@ public void shouldPrepareActiveTaskInStateUpdaterToBeRecycled() {
             final StreamTask activeTaskToRecycle = statefulTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.RESTORING)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(activeTaskToRecycle));
     
    @@ -450,9 +444,9 @@ public void shouldPrepareActiveTaskInStateUpdaterToBeRecycled() {
                 mkMap(mkEntry(activeTaskToRecycle.id(), activeTaskToRecycle.inputPartitions()))
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(tasks).addPendingTaskToRecycle(activeTaskToRecycle.id(), activeTaskToRecycle.inputPartitions());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(tasks).addPendingTaskToRecycle(activeTaskToRecycle.id(), activeTaskToRecycle.inputPartitions());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -460,7 +454,7 @@ public void shouldPrepareStandbyTaskInStateUpdaterToBeRecycled() {
             final StandbyTask standbyTaskToRecycle = standbyTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(standbyTaskToRecycle));
     
    @@ -469,10 +463,10 @@ public void shouldPrepareStandbyTaskInStateUpdaterToBeRecycled() {
                 Collections.emptyMap()
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(stateUpdater).remove(standbyTaskToRecycle.id());
    -        Mockito.verify(tasks).addPendingTaskToRecycle(standbyTaskToRecycle.id(), standbyTaskToRecycle.inputPartitions());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(stateUpdater).remove(standbyTaskToRecycle.id());
    +        verify(tasks).addPendingTaskToRecycle(standbyTaskToRecycle.id(), standbyTaskToRecycle.inputPartitions());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -480,16 +474,16 @@ public void shouldRemoveUnusedActiveTaskFromStateUpdater() {
             final StreamTask activeTaskToClose = statefulTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.RESTORING)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(activeTaskToClose));
     
             taskManager.handleAssignment(Collections.emptyMap(), Collections.emptyMap());
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(stateUpdater).remove(activeTaskToClose.id());
    -        Mockito.verify(tasks).addPendingTaskToCloseClean(activeTaskToClose.id());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(stateUpdater).remove(activeTaskToClose.id());
    +        verify(tasks).addPendingTaskToCloseClean(activeTaskToClose.id());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -497,16 +491,16 @@ public void shouldRemoveUnusedStandbyTaskFromStateUpdater() {
             final StandbyTask standbyTaskToClose = standbyTask(taskId02, taskId02ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId02Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(standbyTaskToClose));
     
             taskManager.handleAssignment(Collections.emptyMap(), Collections.emptyMap());
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(stateUpdater).remove(standbyTaskToClose.id());
    -        Mockito.verify(tasks).addPendingTaskToCloseClean(standbyTaskToClose.id());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(stateUpdater).remove(standbyTaskToClose.id());
    +        verify(tasks).addPendingTaskToCloseClean(standbyTaskToClose.id());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -515,7 +509,7 @@ public void shouldUpdateInputPartitionOfActiveTaskInStateUpdater() {
                 .inState(State.RESTORING)
                 .withInputPartitions(taskId03Partitions).build();
             final Set newInputPartitions = taskId02Partitions;
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(activeTaskToUpdateInputPartitions));
     
    @@ -524,10 +518,10 @@ public void shouldUpdateInputPartitionOfActiveTaskInStateUpdater() {
                 Collections.emptyMap()
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(stateUpdater).remove(activeTaskToUpdateInputPartitions.id());
    -        Mockito.verify(tasks).addPendingTaskToUpdateInputPartitions(activeTaskToUpdateInputPartitions.id(), newInputPartitions);
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(stateUpdater).remove(activeTaskToUpdateInputPartitions.id());
    +        verify(tasks).addPendingTaskToUpdateInputPartitions(activeTaskToUpdateInputPartitions.id(), newInputPartitions);
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -536,7 +530,7 @@ public void shouldCloseReviveAndUpdateInputPartitionOfActiveTaskInStateUpdater()
                 .inState(State.RESTORING)
                 .withInputPartitions(taskId03Partitions).build();
             final Set newInputPartitions = taskId02Partitions;
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(activeTaskToUpdateInputPartitions));
             when(tasks.removePendingTaskToCloseClean(activeTaskToUpdateInputPartitions.id())).thenReturn(true);
    @@ -546,11 +540,11 @@ public void shouldCloseReviveAndUpdateInputPartitionOfActiveTaskInStateUpdater()
                 Collections.emptyMap()
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(stateUpdater, never()).remove(activeTaskToUpdateInputPartitions.id());
    -        Mockito.verify(tasks).removePendingTaskToCloseClean(activeTaskToUpdateInputPartitions.id());
    -        Mockito.verify(tasks).addPendingTaskToCloseReviveAndUpdateInputPartitions(activeTaskToUpdateInputPartitions.id(), newInputPartitions);
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(stateUpdater, never()).remove(activeTaskToUpdateInputPartitions.id());
    +        verify(tasks).removePendingTaskToCloseClean(activeTaskToUpdateInputPartitions.id());
    +        verify(tasks).addPendingTaskToCloseReviveAndUpdateInputPartitions(activeTaskToUpdateInputPartitions.id(), newInputPartitions);
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -558,7 +552,7 @@ public void shouldKeepReassignedActiveTaskInStateUpdater() {
             final StreamTask reassignedActiveTask = statefulTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.RESTORING)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(reassignedActiveTask));
     
    @@ -567,8 +561,8 @@ public void shouldKeepReassignedActiveTaskInStateUpdater() {
                 Collections.emptyMap()
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -576,7 +570,7 @@ public void shouldMoveReassignedSuspendedActiveTaskToStateUpdater() {
             final StreamTask reassignedActiveTask = statefulTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.SUSPENDED)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(tasks.allTasks()).thenReturn(mkSet(reassignedActiveTask));
     
    @@ -585,10 +579,10 @@ public void shouldMoveReassignedSuspendedActiveTaskToStateUpdater() {
                 Collections.emptyMap()
             );
     
    -        Mockito.verify(tasks).removeTask(reassignedActiveTask);
    -        Mockito.verify(stateUpdater).add(reassignedActiveTask);
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(tasks).removeTask(reassignedActiveTask);
    +        verify(stateUpdater).add(reassignedActiveTask);
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -596,7 +590,7 @@ public void shouldRemoveReassignedRevokedActiveTaskInStateUpdaterFromPendingTask
             final StreamTask reassignedRevokedActiveTask = statefulTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.RESTORING)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(reassignedRevokedActiveTask));
     
    @@ -605,9 +599,9 @@ public void shouldRemoveReassignedRevokedActiveTaskInStateUpdaterFromPendingTask
                 Collections.emptyMap()
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(tasks).removePendingActiveTaskToSuspend(reassignedRevokedActiveTask.id());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(tasks).removePendingActiveTaskToSuspend(reassignedRevokedActiveTask.id());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -615,7 +609,7 @@ public void shouldRemoveReassignedLostTaskInStateUpdaterFromPendingTaskToCloseCl
             final StreamTask reassignedLostTask = statefulTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.RESTORING)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(reassignedLostTask));
             when(tasks.removePendingTaskToCloseClean(reassignedLostTask.id())).thenReturn(true);
    @@ -625,10 +619,10 @@ public void shouldRemoveReassignedLostTaskInStateUpdaterFromPendingTaskToCloseCl
                 Collections.emptyMap()
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(tasks).removePendingTaskToCloseClean(reassignedLostTask.id());
    -        Mockito.verify(tasks).addPendingTaskToAddBack(reassignedLostTask.id());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(tasks).removePendingTaskToCloseClean(reassignedLostTask.id());
    +        verify(tasks).addPendingTaskToAddBack(reassignedLostTask.id());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -636,7 +630,7 @@ public void shouldRemoveReassignedTaskInStateUpdaterFromPendingSuspend() {
             final StreamTask reassignedTask = statefulTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.RESTORING)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(reassignedTask));
             when(tasks.removePendingActiveTaskToSuspend(reassignedTask.id())).thenReturn(true);
    @@ -646,10 +640,10 @@ public void shouldRemoveReassignedTaskInStateUpdaterFromPendingSuspend() {
                 Collections.emptyMap()
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(tasks).removePendingActiveTaskToSuspend(reassignedTask.id());
    -        Mockito.verify(tasks).addPendingTaskToAddBack(reassignedTask.id());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(tasks).removePendingActiveTaskToSuspend(reassignedTask.id());
    +        verify(tasks).addPendingTaskToAddBack(reassignedTask.id());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -657,7 +651,7 @@ public void shouldReAddStandbyTaskFromPendingRecycle() {
             final StandbyTask reassignedStandbyTask = standbyTask(taskId01, taskId01ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId01Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(reassignedStandbyTask));
             when(tasks.removePendingTaskToRecycle(reassignedStandbyTask.id())).thenReturn(taskId01Partitions);
    @@ -667,10 +661,10 @@ public void shouldReAddStandbyTaskFromPendingRecycle() {
                 mkMap(mkEntry(reassignedStandbyTask.id(), reassignedStandbyTask.inputPartitions()))
             );
     
    -        Mockito.verify(tasks).removePendingTaskToRecycle(reassignedStandbyTask.id());
    -        Mockito.verify(tasks).addPendingTaskToAddBack(reassignedStandbyTask.id());
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(tasks).removePendingTaskToRecycle(reassignedStandbyTask.id());
    +        verify(tasks).addPendingTaskToAddBack(reassignedStandbyTask.id());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -679,7 +673,7 @@ public void shouldNeverUpdateInputPartitionsOfStandbyTaskInStateUpdater() {
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId02Partitions).build();
             final Set newInputPartitions = taskId03Partitions;
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(standbyTaskToUpdateInputPartitions));
     
    @@ -688,11 +682,11 @@ public void shouldNeverUpdateInputPartitionsOfStandbyTaskInStateUpdater() {
                 mkMap(mkEntry(standbyTaskToUpdateInputPartitions.id(), newInputPartitions))
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(stateUpdater, never()).remove(standbyTaskToUpdateInputPartitions.id());
    -        Mockito.verify(tasks, never())
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(stateUpdater, never()).remove(standbyTaskToUpdateInputPartitions.id());
    +        verify(tasks, never())
                 .addPendingTaskToUpdateInputPartitions(standbyTaskToUpdateInputPartitions.id(), newInputPartitions);
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -701,7 +695,7 @@ public void shouldNeverCloseReviveAndUpdateInputPartitionsOfStandbyTaskInStateUp
                     .inState(State.RUNNING)
                     .withInputPartitions(taskId02Partitions).build();
             final Set newInputPartitions = taskId03Partitions;
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(standbyTaskToUpdateInputPartitions));
     
    @@ -710,12 +704,12 @@ public void shouldNeverCloseReviveAndUpdateInputPartitionsOfStandbyTaskInStateUp
                     mkMap(mkEntry(standbyTaskToUpdateInputPartitions.id(), newInputPartitions))
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(stateUpdater, never()).remove(standbyTaskToUpdateInputPartitions.id());
    -        Mockito.verify(tasks, never()).removePendingTaskToCloseClean(standbyTaskToUpdateInputPartitions.id());
    -        Mockito.verify(tasks, never())
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(stateUpdater, never()).remove(standbyTaskToUpdateInputPartitions.id());
    +        verify(tasks, never()).removePendingTaskToCloseClean(standbyTaskToUpdateInputPartitions.id());
    +        verify(tasks, never())
                     .addPendingTaskToCloseReviveAndUpdateInputPartitions(standbyTaskToUpdateInputPartitions.id(), newInputPartitions);
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -723,7 +717,7 @@ public void shouldKeepReassignedStandbyTaskInStateUpdater() {
             final StandbyTask reassignedStandbyTask = standbyTask(taskId02, taskId02ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId02Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(reassignedStandbyTask));
     
    @@ -732,8 +726,8 @@ public void shouldKeepReassignedStandbyTaskInStateUpdater() {
                 mkMap(mkEntry(reassignedStandbyTask.id(), reassignedStandbyTask.inputPartitions()))
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -744,7 +738,7 @@ public void shouldAssignMultipleTasksInStateUpdater() {
             final StandbyTask standbyTaskToRecycle = standbyTask(taskId02, taskId02ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId02Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(activeTaskToClose, standbyTaskToRecycle));
     
    @@ -753,12 +747,12 @@ public void shouldAssignMultipleTasksInStateUpdater() {
                 Collections.emptyMap()
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(stateUpdater).remove(activeTaskToClose.id());
    -        Mockito.verify(tasks).addPendingTaskToCloseClean(activeTaskToClose.id());
    -        Mockito.verify(stateUpdater).remove(standbyTaskToRecycle.id());
    -        Mockito.verify(tasks).addPendingTaskToRecycle(standbyTaskToRecycle.id(), standbyTaskToRecycle.inputPartitions());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(stateUpdater).remove(activeTaskToClose.id());
    +        verify(tasks).addPendingTaskToCloseClean(activeTaskToClose.id());
    +        verify(stateUpdater).remove(standbyTaskToRecycle.id());
    +        verify(tasks).addPendingTaskToRecycle(standbyTaskToRecycle.id(), standbyTaskToRecycle.inputPartitions());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -769,7 +763,7 @@ public void shouldReturnStateUpdaterTasksInAllTasks() {
             final StandbyTask standbyTask = standbyTask(taskId02, taskId02ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId02Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
     
             when(stateUpdater.getTasks()).thenReturn(mkSet(standbyTask));
    @@ -785,10 +779,9 @@ public void shouldNotReturnStateUpdaterTasksInOwnedTasks() {
             final StandbyTask standbyTask = standbyTask(taskId02, taskId02ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId02Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
     
    -        when(stateUpdater.getTasks()).thenReturn(mkSet(standbyTask));
             when(tasks.allTasksPerId()).thenReturn(mkMap(mkEntry(taskId03, activeTask)));
             assertEquals(taskManager.allOwnedTasks(), mkMap(mkEntry(taskId03, activeTask)));
         }
    @@ -798,7 +791,7 @@ public void shouldCreateActiveTaskDuringAssignment() {
             final StreamTask activeTaskToBeCreated = statefulTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.CREATED)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             final Set createdTasks = mkSet(activeTaskToBeCreated);
             final Map> tasksToBeCreated = mkMap(
    @@ -807,8 +800,8 @@ public void shouldCreateActiveTaskDuringAssignment() {
     
             taskManager.handleAssignment(tasksToBeCreated, Collections.emptyMap());
     
    -        Mockito.verify(tasks).addPendingTasksToInit(createdTasks);
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(tasks).addPendingTasksToInit(createdTasks);
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -816,7 +809,7 @@ public void shouldCreateStandbyTaskDuringAssignment() {
             final StandbyTask standbyTaskToBeCreated = standbyTask(taskId02, taskId02ChangelogPartitions)
                 .inState(State.CREATED)
                 .withInputPartitions(taskId02Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             final Set createdTasks = mkSet(standbyTaskToBeCreated);
             when(standbyTaskCreator.createTasks(mkMap(
    @@ -828,8 +821,8 @@ public void shouldCreateStandbyTaskDuringAssignment() {
                 mkMap(mkEntry(standbyTaskToBeCreated.id(), standbyTaskToBeCreated.inputPartitions()))
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(tasks).addPendingTasksToInit(createdTasks);
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(tasks).addPendingTasksToInit(createdTasks);
         }
     
         @Test
    @@ -848,12 +841,12 @@ public void shouldAddRecycledStandbyTaskfromActiveToPendingTasksToInitWithStateU
     
             taskManager.handleAssignment(emptyMap(), mkMap(mkEntry(taskId01, taskId01Partitions)));
     
    -        Mockito.verify(activeTaskToRecycle).prepareCommit();
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(activeTaskToRecycle.id());
    -        Mockito.verify(tasks).addPendingTasksToInit(mkSet(standbyTask));
    -        Mockito.verify(tasks).removeTask(activeTaskToRecycle);
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskToRecycle).prepareCommit();
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(activeTaskToRecycle.id());
    +        verify(tasks).addPendingTasksToInit(mkSet(standbyTask));
    +        verify(tasks).removeTask(activeTaskToRecycle);
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -872,11 +865,11 @@ public void shouldAddRecycledStandbyTaskfromActiveToTaskRegistryWithStateUpdater
     
             taskManager.handleAssignment(emptyMap(), mkMap(mkEntry(taskId01, taskId01Partitions)));
     
    -        Mockito.verify(activeTaskToRecycle).prepareCommit();
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(activeTaskToRecycle.id());
    -        Mockito.verify(tasks).replaceActiveWithStandby(standbyTask);
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskToRecycle).prepareCommit();
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(activeTaskToRecycle.id());
    +        verify(tasks).replaceActiveWithStandby(standbyTask);
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -884,7 +877,7 @@ public void shouldThrowDuringAssignmentIfStandbyTaskToRecycleIsFoundInTasksRegis
             final StandbyTask standbyTaskToRecycle = standbyTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             when(tasks.allTasks()).thenReturn(mkSet(standbyTaskToRecycle));
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
     
    @@ -898,7 +891,7 @@ public void shouldThrowDuringAssignmentIfStandbyTaskToRecycleIsFoundInTasksRegis
     
             assertEquals(illegalStateException.getMessage(), "Standby tasks should only be managed by the state updater, " +
                 "but standby task " + taskId03 + " is managed by the stream thread");
    -        Mockito.verifyNoInteractions(activeTaskCreator);
    +        verifyNoInteractions(activeTaskCreator);
         }
     
         @Test
    @@ -906,18 +899,18 @@ public void shouldAssignActiveTaskInTasksRegistryToBeClosedCleanlyWithStateUpdat
             final StreamTask activeTaskToClose = statefulTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(tasks.allTasks()).thenReturn(mkSet(activeTaskToClose));
     
             taskManager.handleAssignment(Collections.emptyMap(), Collections.emptyMap());
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(activeTaskToClose.id());
    -        Mockito.verify(activeTaskToClose).prepareCommit();
    -        Mockito.verify(activeTaskToClose).closeClean();
    -        Mockito.verify(tasks).removeTask(activeTaskToClose);
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(activeTaskToClose.id());
    +        verify(activeTaskToClose).prepareCommit();
    +        verify(activeTaskToClose).closeClean();
    +        verify(tasks).removeTask(activeTaskToClose);
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -925,7 +918,7 @@ public void shouldThrowDuringAssignmentIfStandbyTaskToCloseIsFoundInTasksRegistr
             final StandbyTask standbyTaskToClose = standbyTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(tasks.allTasks()).thenReturn(mkSet(standbyTaskToClose));
     
    @@ -936,7 +929,7 @@ public void shouldThrowDuringAssignmentIfStandbyTaskToCloseIsFoundInTasksRegistr
     
             assertEquals(illegalStateException.getMessage(), "Standby tasks should only be managed by the state updater, " +
                 "but standby task " + taskId03 + " is managed by the stream thread");
    -        Mockito.verifyNoInteractions(activeTaskCreator);
    +        verifyNoInteractions(activeTaskCreator);
         }
     
         @Test
    @@ -945,7 +938,7 @@ public void shouldAssignActiveTaskInTasksRegistryToUpdateInputPartitionsWithStat
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId03Partitions).build();
             final Set newInputPartitions = taskId02Partitions;
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(tasks.allTasks()).thenReturn(mkSet(activeTaskToUpdateInputPartitions));
             when(tasks.updateActiveTaskInputPartitions(activeTaskToUpdateInputPartitions, newInputPartitions)).thenReturn(true);
    @@ -955,9 +948,9 @@ public void shouldAssignActiveTaskInTasksRegistryToUpdateInputPartitionsWithStat
                 Collections.emptyMap()
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(activeTaskToUpdateInputPartitions).updateInputPartitions(Mockito.eq(newInputPartitions), any());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(activeTaskToUpdateInputPartitions).updateInputPartitions(eq(newInputPartitions), any());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -965,7 +958,7 @@ public void shouldResumeActiveRunningTaskInTasksRegistryWithStateUpdaterEnabled(
             final StreamTask activeTaskToResume = statefulTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(tasks.allTasks()).thenReturn(mkSet(activeTaskToResume));
     
    @@ -974,8 +967,8 @@ public void shouldResumeActiveRunningTaskInTasksRegistryWithStateUpdaterEnabled(
                 Collections.emptyMap()
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -983,7 +976,7 @@ public void shouldResumeActiveSuspendedTaskInTasksRegistryAndAddToStateUpdater()
             final StreamTask activeTaskToResume = statefulTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.SUSPENDED)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(tasks.allTasks()).thenReturn(mkSet(activeTaskToResume));
     
    @@ -992,11 +985,11 @@ public void shouldResumeActiveSuspendedTaskInTasksRegistryAndAddToStateUpdater()
                 Collections.emptyMap()
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    -        Mockito.verify(activeTaskToResume).resume();
    -        Mockito.verify(stateUpdater).add(activeTaskToResume);
    -        Mockito.verify(tasks).removeTask(activeTaskToResume);
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
    +        verify(activeTaskToResume).resume();
    +        verify(stateUpdater).add(activeTaskToResume);
    +        verify(tasks).removeTask(activeTaskToResume);
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -1005,7 +998,7 @@ public void shouldThrowDuringAssignmentIfStandbyTaskToUpdateInputPartitionsIsFou
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId02Partitions).build();
             final Set newInputPartitions = taskId03Partitions;
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(tasks.allTasks()).thenReturn(mkSet(standbyTaskToUpdateInputPartitions));
     
    @@ -1019,7 +1012,7 @@ public void shouldThrowDuringAssignmentIfStandbyTaskToUpdateInputPartitionsIsFou
     
             assertEquals(illegalStateException.getMessage(), "Standby tasks should only be managed by the state updater, " +
                 "but standby task " + taskId02 + " is managed by the stream thread");
    -        Mockito.verifyNoInteractions(activeTaskCreator);
    +        verifyNoInteractions(activeTaskCreator);
         }
     
         @Test
    @@ -1030,7 +1023,7 @@ public void shouldAssignMultipleTasksInTasksRegistryWithStateUpdaterEnabled() {
             final StreamTask activeTaskToCreate = statefulTask(taskId02, taskId02ChangelogPartitions)
                 .inState(State.CREATED)
                 .withInputPartitions(taskId02Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(tasks.allTasks()).thenReturn(mkSet(activeTaskToClose));
     
    @@ -1039,12 +1032,12 @@ public void shouldAssignMultipleTasksInTasksRegistryWithStateUpdaterEnabled() {
                 Collections.emptyMap()
             );
     
    -        Mockito.verify(activeTaskCreator).createTasks(
    +        verify(activeTaskCreator).createTasks(
                 consumer,
                 mkMap(mkEntry(activeTaskToCreate.id(), activeTaskToCreate.inputPartitions()))
             );
    -        Mockito.verify(activeTaskToClose).closeClean();
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verify(activeTaskToClose).closeClean();
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
         }
     
         @Test
    @@ -1061,10 +1054,10 @@ public void shouldAddTasksToStateUpdater() {
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(task00).initializeIfNeeded();
    -        Mockito.verify(task01).initializeIfNeeded();
    -        Mockito.verify(stateUpdater).add(task00);
    -        Mockito.verify(stateUpdater).add(task01);
    +        verify(task00).initializeIfNeeded();
    +        verify(task01).initializeIfNeeded();
    +        verify(stateUpdater).add(task00);
    +        verify(stateUpdater).add(task01);
         }
     
         @Test
    @@ -1083,13 +1076,13 @@ public void shouldRetryInitializationWhenLockExceptionInStateUpdater() {
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(task00).initializeIfNeeded();
    -        Mockito.verify(task01).initializeIfNeeded();
    -        Mockito.verify(tasks).addPendingTasksToInit(
    -            Mockito.argThat(tasksToInit -> tasksToInit.contains(task00) && !tasksToInit.contains(task01))
    +        verify(task00).initializeIfNeeded();
    +        verify(task01).initializeIfNeeded();
    +        verify(tasks).addPendingTasksToInit(
    +            argThat(tasksToInit -> tasksToInit.contains(task00) && !tasksToInit.contains(task01))
             );
    -        Mockito.verify(stateUpdater, never()).add(task00);
    -        Mockito.verify(stateUpdater).add(task01);
    +        verify(stateUpdater, never()).add(task00);
    +        verify(stateUpdater).add(task01);
         }
     
         @Test
    @@ -1120,13 +1113,13 @@ public void shouldRetryInitializationWhenLockExceptionAfterRecyclingInStateUpdat
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(task00Converted).initializeIfNeeded();
    -        Mockito.verify(task01Converted).initializeIfNeeded();
    -        Mockito.verify(tasks).addPendingTasksToInit(
    -            Mockito.argThat(tasksToInit -> tasksToInit.contains(task00Converted) && !tasksToInit.contains(task01Converted))
    +        verify(task00Converted).initializeIfNeeded();
    +        verify(task01Converted).initializeIfNeeded();
    +        verify(tasks).addPendingTasksToInit(
    +            argThat(tasksToInit -> tasksToInit.contains(task00Converted) && !tasksToInit.contains(task01Converted))
             );
    -        Mockito.verify(stateUpdater, never()).add(task00Converted);
    -        Mockito.verify(stateUpdater).add(task01Converted);
    +        verify(stateUpdater, never()).add(task00Converted);
    +        verify(stateUpdater).add(task01Converted);
         }
     
         @Test
    @@ -1154,13 +1147,13 @@ public void shouldRecycleTasksRemovedFromStateUpdater() {
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(any());
    -        Mockito.verify(task00).suspend();
    -        Mockito.verify(task01).suspend();
    -        Mockito.verify(task00Converted).initializeIfNeeded();
    -        Mockito.verify(task01Converted).initializeIfNeeded();
    -        Mockito.verify(stateUpdater).add(task00Converted);
    -        Mockito.verify(stateUpdater).add(task01Converted);
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(any());
    +        verify(task00).suspend();
    +        verify(task01).suspend();
    +        verify(task00Converted).initializeIfNeeded();
    +        verify(task01Converted).initializeIfNeeded();
    +        verify(stateUpdater).add(task00Converted);
    +        verify(stateUpdater).add(task01Converted);
         }
     
         @Test
    @@ -1181,11 +1174,11 @@ public void shouldCloseTasksRemovedFromStateUpdater() {
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(any());
    -        Mockito.verify(task00).suspend();
    -        Mockito.verify(task00).closeClean();
    -        Mockito.verify(task01).suspend();
    -        Mockito.verify(task01).closeClean();
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(any());
    +        verify(task00).suspend();
    +        verify(task00).closeClean();
    +        verify(task01).suspend();
    +        verify(task01).closeClean();
         }
     
         @Test
    @@ -1207,14 +1200,14 @@ public void shouldUpdateInputPartitionsOfTasksRemovedFromStateUpdater() {
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(activeTask).updateInputPartitions(Mockito.eq(taskId02Partitions), anyMap());
    -        Mockito.verify(activeTask, never()).closeDirty();
    -        Mockito.verify(activeTask, never()).closeClean();
    -        Mockito.verify(stateUpdater).add(activeTask);
    -        Mockito.verify(standbyTask).updateInputPartitions(Mockito.eq(taskId03Partitions), anyMap());
    -        Mockito.verify(standbyTask, never()).closeDirty();
    -        Mockito.verify(standbyTask, never()).closeClean();
    -        Mockito.verify(stateUpdater).add(standbyTask);
    +        verify(activeTask).updateInputPartitions(eq(taskId02Partitions), anyMap());
    +        verify(activeTask, never()).closeDirty();
    +        verify(activeTask, never()).closeClean();
    +        verify(stateUpdater).add(activeTask);
    +        verify(standbyTask).updateInputPartitions(eq(taskId03Partitions), anyMap());
    +        verify(standbyTask, never()).closeDirty();
    +        verify(standbyTask, never()).closeClean();
    +        verify(stateUpdater).add(standbyTask);
         }
     
         @Test
    @@ -1231,12 +1224,12 @@ public void shouldCloseReviveAndUpdateInputPartitionsOfTasksRemovedFromStateUpda
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(activeTask).closeClean();
    -        Mockito.verify(activeTask).revive();
    -        Mockito.verify(activeTask).updateInputPartitions(Mockito.eq(taskId02Partitions), anyMap());
    -        Mockito.verify(activeTask).initializeIfNeeded();
    -        Mockito.verify(activeTask, never()).closeDirty();
    -        Mockito.verify(stateUpdater).add(activeTask);
    +        verify(activeTask).closeClean();
    +        verify(activeTask).revive();
    +        verify(activeTask).updateInputPartitions(eq(taskId02Partitions), anyMap());
    +        verify(activeTask).initializeIfNeeded();
    +        verify(activeTask, never()).closeDirty();
    +        verify(stateUpdater).add(activeTask);
         }
     
         @Test
    @@ -1252,13 +1245,12 @@ public void shouldSuspendRevokedTaskRemovedFromStateUpdater() {
             when(stateUpdater.hasRemovedTasks()).thenReturn(true);
             when(stateUpdater.drainRemovedTasks()).thenReturn(mkSet(statefulTask));
             taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
    -        taskManager.setMainConsumer(mockitoConsumer);
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(statefulTask).suspend();
    -        Mockito.verify(tasks).addTask(statefulTask);
    -        Mockito.verifyNoInteractions(mockitoConsumer);
    +        verify(statefulTask).suspend();
    +        verify(tasks).addTask(statefulTask);
    +        verifyNoInteractions(consumer);
         }
     
         @Test
    @@ -1284,7 +1276,7 @@ public void shouldHandleMultipleRemovedTasksFromStateUpdater() {
             when(stateUpdater.drainRemovedTasks())
                 .thenReturn(mkSet(taskToRecycle0, taskToRecycle1, taskToClose, taskToUpdateInputPartitions, taskToCloseReviveAndUpdateInputPartitions));
             when(stateUpdater.restoresActiveTasks()).thenReturn(true);
    -        when(activeTaskCreator.createActiveTaskFromStandby(taskToRecycle1, taskId01Partitions, mockitoConsumer))
    +        when(activeTaskCreator.createActiveTaskFromStandby(taskToRecycle1, taskId01Partitions, consumer))
                 .thenReturn(convertedTask1);
             when(standbyTaskCreator.createStandbyTaskFromActive(taskToRecycle0, taskId00Partitions))
                 .thenReturn(convertedTask0);
    @@ -1302,25 +1294,24 @@ public void shouldHandleMultipleRemovedTasksFromStateUpdater() {
                 argThat(taskId -> !taskId.equals(taskToCloseReviveAndUpdateInputPartitions.id()))
             )).thenReturn(null);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
    -        taskManager.setMainConsumer(mockitoConsumer);
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter -> { });
     
    -        Mockito.verify(activeTaskCreator, times(3)).closeAndRemoveTaskProducerIfNeeded(any());
    -        Mockito.verify(activeTaskCreator, times(3)).closeAndRemoveTaskProducerIfNeeded(any());
    -        Mockito.verify(convertedTask0).initializeIfNeeded();
    -        Mockito.verify(convertedTask1).initializeIfNeeded();
    -        Mockito.verify(stateUpdater).add(convertedTask0);
    -        Mockito.verify(stateUpdater).add(convertedTask1);
    -        Mockito.verify(taskToClose).closeClean();
    -        Mockito.verify(taskToUpdateInputPartitions).updateInputPartitions(Mockito.eq(taskId04Partitions), anyMap());
    -        Mockito.verify(stateUpdater).add(taskToUpdateInputPartitions);
    -        Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).closeClean();
    -        Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).revive();
    -        Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).updateInputPartitions(Mockito.eq(taskId05Partitions), anyMap());
    -        Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).initializeIfNeeded();
    -        Mockito.verify(stateUpdater).add(taskToCloseReviveAndUpdateInputPartitions);
    -        Mockito.verifyNoInteractions(mockitoConsumer);
    +        verify(activeTaskCreator, times(3)).closeAndRemoveTaskProducerIfNeeded(any());
    +        verify(activeTaskCreator, times(3)).closeAndRemoveTaskProducerIfNeeded(any());
    +        verify(convertedTask0).initializeIfNeeded();
    +        verify(convertedTask1).initializeIfNeeded();
    +        verify(stateUpdater).add(convertedTask0);
    +        verify(stateUpdater).add(convertedTask1);
    +        verify(taskToClose).closeClean();
    +        verify(taskToUpdateInputPartitions).updateInputPartitions(eq(taskId04Partitions), anyMap());
    +        verify(stateUpdater).add(taskToUpdateInputPartitions);
    +        verify(taskToCloseReviveAndUpdateInputPartitions).closeClean();
    +        verify(taskToCloseReviveAndUpdateInputPartitions).revive();
    +        verify(taskToCloseReviveAndUpdateInputPartitions).updateInputPartitions(eq(taskId05Partitions), anyMap());
    +        verify(taskToCloseReviveAndUpdateInputPartitions).initializeIfNeeded();
    +        verify(stateUpdater).add(taskToCloseReviveAndUpdateInputPartitions);
    +        verifyNoInteractions(consumer);
         }
     
         @Test
    @@ -1369,8 +1360,8 @@ public void shouldAddActiveTaskWithRevokedInputPartitionsInStateUpdaterToPending
     
             taskManager.handleRevocation(task.inputPartitions());
     
    -        Mockito.verify(tasks).addPendingActiveTaskToSuspend(task.id());
    -        Mockito.verify(stateUpdater, never()).remove(task.id());
    +        verify(tasks).addPendingActiveTaskToSuspend(task.id());
    +        verify(stateUpdater, never()).remove(task.id());
         }
     
         public void shouldAddMultipleActiveTasksWithRevokedInputPartitionsInStateUpdaterToPendingTasksToSuspend() {
    @@ -1385,8 +1376,8 @@ public void shouldAddMultipleActiveTasksWithRevokedInputPartitionsInStateUpdater
     
             taskManager.handleRevocation(union(HashSet::new, taskId00Partitions, taskId01Partitions));
     
    -        Mockito.verify(tasks).addPendingActiveTaskToSuspend(task1.id());
    -        Mockito.verify(tasks).addPendingActiveTaskToSuspend(task2.id());
    +        verify(tasks).addPendingActiveTaskToSuspend(task1.id());
    +        verify(tasks).addPendingActiveTaskToSuspend(task2.id());
         }
     
         @Test
    @@ -1399,8 +1390,8 @@ public void shouldNotAddActiveTaskWithoutRevokedInputPartitionsInStateUpdaterToP
     
             taskManager.handleRevocation(taskId01Partitions);
     
    -        Mockito.verify(stateUpdater, never()).remove(task.id());
    -        Mockito.verify(tasks, never()).addPendingActiveTaskToSuspend(task.id());
    +        verify(stateUpdater, never()).remove(task.id());
    +        verify(tasks, never()).addPendingActiveTaskToSuspend(task.id());
         }
     
         @Test
    @@ -1413,8 +1404,8 @@ public void shouldNotRevokeStandbyTaskInStateUpdaterOnRevocation() {
     
             taskManager.handleRevocation(taskId00Partitions);
     
    -        Mockito.verify(stateUpdater, never()).remove(task.id());
    -        Mockito.verify(tasks, never()).addPendingActiveTaskToSuspend(task.id());
    +        verify(stateUpdater, never()).remove(task.id());
    +        verify(tasks, never()).addPendingActiveTaskToSuspend(task.id());
         }
     
         @Test
    @@ -1433,12 +1424,12 @@ public void shouldRemoveAllActiveTasksFromStateUpdaterOnPartitionLost() {
     
             taskManager.handleLostAll();
     
    -        Mockito.verify(stateUpdater).remove(task1.id());
    -        Mockito.verify(stateUpdater, never()).remove(task2.id());
    -        Mockito.verify(stateUpdater).remove(task3.id());
    -        Mockito.verify(tasks).addPendingTaskToCloseClean(task1.id());
    -        Mockito.verify(tasks, never()).addPendingTaskToCloseClean(task2.id());
    -        Mockito.verify(tasks).addPendingTaskToCloseClean(task3.id());
    +        verify(stateUpdater).remove(task1.id());
    +        verify(stateUpdater, never()).remove(task2.id());
    +        verify(stateUpdater).remove(task3.id());
    +        verify(tasks).addPendingTaskToCloseClean(task1.id());
    +        verify(tasks, never()).addPendingTaskToCloseClean(task2.id());
    +        verify(tasks).addPendingTaskToCloseClean(task3.id());
         }
     
         private TaskManager setupForRevocationAndLost(final Set tasksInStateUpdater,
    @@ -1456,14 +1447,13 @@ public void shouldTransitRestoredTaskToRunning() {
                 .withInputPartitions(taskId00Partitions).build();
             final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTransitionToRunningOfRestoredTask(task, tasks);
    -        taskManager.setMainConsumer(mockitoConsumer);
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(task).completeRestoration(noOpResetter);
    -        Mockito.verify(task).clearTaskTimeout();
    -        Mockito.verify(tasks).addTask(task);
    -        Mockito.verify(mockitoConsumer).resume(task.inputPartitions());
    +        verify(task).completeRestoration(noOpResetter);
    +        verify(task).clearTaskTimeout();
    +        verify(tasks).addTask(task);
    +        verify(consumer).resume(task.inputPartitions());
         }
     
         @Test
    @@ -1473,16 +1463,15 @@ public void shouldHandleTimeoutExceptionInTransitRestoredTaskToRunning() {
                 .withInputPartitions(taskId00Partitions).build();
             final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTransitionToRunningOfRestoredTask(task, tasks);
    -        taskManager.setMainConsumer(mockitoConsumer);
             final TimeoutException timeoutException = new TimeoutException();
             doThrow(timeoutException).when(task).completeRestoration(noOpResetter);
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(task).maybeInitTaskTimeoutOrThrow(anyLong(), Mockito.eq(timeoutException));
    -        Mockito.verify(tasks, never()).addTask(task);
    -        Mockito.verify(task, never()).clearTaskTimeout();
    -        Mockito.verifyNoInteractions(mockitoConsumer);
    +        verify(task).maybeInitTaskTimeoutOrThrow(anyLong(), eq(timeoutException));
    +        verify(tasks, never()).addTask(task);
    +        verify(task, never()).clearTaskTimeout();
    +        verifyNoInteractions(consumer);
         }
     
         private TaskManager setUpTransitionToRunningOfRestoredTask(final StreamTask statefulTask,
    @@ -1521,10 +1510,10 @@ public void shouldRecycleRestoredTask() {
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(statefulTask.id());
    -        Mockito.verify(statefulTask).suspend();
    -        Mockito.verify(standbyTask).initializeIfNeeded();
    -        Mockito.verify(stateUpdater).add(standbyTask);
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(statefulTask.id());
    +        verify(statefulTask).suspend();
    +        verify(standbyTask).initializeIfNeeded();
    +        verify(stateUpdater).add(standbyTask);
         }
     
         @Test
    @@ -1541,8 +1530,8 @@ public void shouldHandleExceptionThrownDuringConversionInRecycleRestoredTask() {
                 () -> taskManager.checkStateUpdater(time.milliseconds(), noOpResetter)
             );
     
    -        Mockito.verify(stateUpdater, never()).add(any());
    -        Mockito.verify(statefulTask).closeDirty();
    +        verify(stateUpdater, never()).add(any());
    +        verify(statefulTask).closeDirty();
         }
     
         @Test
    @@ -1563,8 +1552,8 @@ public void shouldHandleExceptionThrownDuringTaskInitInRecycleRestoredTask() {
                 () -> taskManager.checkStateUpdater(time.milliseconds(), noOpResetter)
             );
     
    -        Mockito.verify(stateUpdater, never()).add(any());
    -        Mockito.verify(standbyTask).closeDirty();
    +        verify(stateUpdater, never()).add(any());
    +        verify(standbyTask).closeDirty();
         }
     
         private TaskManager setUpRecycleRestoredTask(final StreamTask statefulTask) {
    @@ -1586,11 +1575,11 @@ public void shouldCloseCleanRestoredTask() {
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(statefulTask.id());
    -        Mockito.verify(statefulTask).suspend();
    -        Mockito.verify(statefulTask).closeClean();
    -        Mockito.verify(statefulTask, never()).closeDirty();
    -        Mockito.verify(tasks, never()).removeTask(statefulTask);
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(statefulTask.id());
    +        verify(statefulTask).suspend();
    +        verify(statefulTask).closeClean();
    +        verify(statefulTask, never()).closeDirty();
    +        verify(tasks, never()).removeTask(statefulTask);
         }
     
         @Test
    @@ -1607,9 +1596,9 @@ public void shouldHandleExceptionThrownDuringCloseInCloseCleanRestoredTask() {
                 () -> taskManager.checkStateUpdater(time.milliseconds(), noOpResetter)
             );
     
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(statefulTask.id());
    -        Mockito.verify(statefulTask).closeDirty();
    -        Mockito.verify(tasks, never()).removeTask(statefulTask);
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(statefulTask.id());
    +        verify(statefulTask).closeDirty();
    +        verify(tasks, never()).removeTask(statefulTask);
         }
     
         @Test
    @@ -1628,8 +1617,8 @@ public void shouldHandleExceptionThrownDuringClosingTaskProducerInCloseCleanRest
                 () -> taskManager.checkStateUpdater(time.milliseconds(), noOpResetter)
             );
     
    -        Mockito.verify(statefulTask, never()).closeDirty();
    -        Mockito.verify(tasks, never()).removeTask(statefulTask);
    +        verify(statefulTask, never()).closeDirty();
    +        verify(tasks, never()).removeTask(statefulTask);
         }
     
         private TaskManager setUpCloseCleanRestoredTask(final StreamTask statefulTask,
    @@ -1656,8 +1645,8 @@ public void shouldAddBackRestoredTask() {
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(stateUpdater).add(statefulTask);
    -        Mockito.verify(tasks, never()).removeTask(statefulTask);
    +        verify(stateUpdater).add(statefulTask);
    +        verify(tasks, never()).removeTask(statefulTask);
         }
     
         @Test
    @@ -1672,15 +1661,14 @@ public void shouldUpdateInputPartitionsOfRestoredTask() {
             when(stateUpdater.drainRestoredActiveTasks(any(Duration.class))).thenReturn(mkSet(statefulTask));
             when(stateUpdater.restoresActiveTasks()).thenReturn(true);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
    -        taskManager.setMainConsumer(mockitoConsumer);
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(mockitoConsumer).resume(statefulTask.inputPartitions());
    -        Mockito.verify(statefulTask).updateInputPartitions(Mockito.eq(taskId01Partitions), anyMap());
    -        Mockito.verify(statefulTask).completeRestoration(noOpResetter);
    -        Mockito.verify(statefulTask).clearTaskTimeout();
    -        Mockito.verify(tasks).addTask(statefulTask);
    +        verify(consumer).resume(statefulTask.inputPartitions());
    +        verify(statefulTask).updateInputPartitions(eq(taskId01Partitions), anyMap());
    +        verify(statefulTask).completeRestoration(noOpResetter);
    +        verify(statefulTask).clearTaskTimeout();
    +        verify(tasks).addTask(statefulTask);
         }
     
         @Test
    @@ -1697,11 +1685,11 @@ public void shouldCloseReviveAndUpdateInputPartitionsOfRestoredTask() {
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(statefulTask).updateInputPartitions(Mockito.eq(taskId01Partitions), anyMap());
    -        Mockito.verify(statefulTask).closeClean();
    -        Mockito.verify(statefulTask).revive();
    -        Mockito.verify(statefulTask).initializeIfNeeded();
    -        Mockito.verify(stateUpdater).add(statefulTask);
    +        verify(statefulTask).updateInputPartitions(eq(taskId01Partitions), anyMap());
    +        verify(statefulTask).closeClean();
    +        verify(statefulTask).revive();
    +        verify(statefulTask).initializeIfNeeded();
    +        verify(stateUpdater).add(statefulTask);
         }
     
         @Test
    @@ -1717,13 +1705,12 @@ public void shouldSuspendRestoredTaskIfRevoked() {
             when(stateUpdater.drainRestoredActiveTasks(any(Duration.class))).thenReturn(mkSet(statefulTask));
             when(stateUpdater.restoresActiveTasks()).thenReturn(true);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
    -        taskManager.setMainConsumer(mockitoConsumer);
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(statefulTask).suspend();
    -        Mockito.verify(tasks).addTask(statefulTask);
    -        Mockito.verifyNoInteractions(mockitoConsumer);
    +        verify(statefulTask).suspend();
    +        verify(tasks).addTask(statefulTask);
    +        verifyNoInteractions(consumer);
         }
     
         @Test
    @@ -1785,16 +1772,16 @@ public void shouldHandleMultipleRestoredTasks() {
     
             taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
     
    -        Mockito.verify(tasks).addTask(taskToTransitToRunning);
    -        Mockito.verify(stateUpdater).add(recycledStandbyTask);
    -        Mockito.verify(stateUpdater).add(recycledStandbyTask);
    -        Mockito.verify(taskToCloseClean).closeClean();
    -        Mockito.verify(stateUpdater).add(taskToAddBack);
    -        Mockito.verify(taskToUpdateInputPartitions).updateInputPartitions(Mockito.eq(taskId05Partitions), anyMap());
    -        Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).closeClean();
    -        Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).revive();
    -        Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).initializeIfNeeded();
    -        Mockito.verify(stateUpdater).add(taskToCloseReviveAndUpdateInputPartitions);
    +        verify(tasks).addTask(taskToTransitToRunning);
    +        verify(stateUpdater).add(recycledStandbyTask);
    +        verify(stateUpdater).add(recycledStandbyTask);
    +        verify(taskToCloseClean).closeClean();
    +        verify(stateUpdater).add(taskToAddBack);
    +        verify(taskToUpdateInputPartitions).updateInputPartitions(eq(taskId05Partitions), anyMap());
    +        verify(taskToCloseReviveAndUpdateInputPartitions).closeClean();
    +        verify(taskToCloseReviveAndUpdateInputPartitions).revive();
    +        verify(taskToCloseReviveAndUpdateInputPartitions).initializeIfNeeded();
    +        verify(stateUpdater).add(taskToCloseReviveAndUpdateInputPartitions);
         }
     
         @Test
    @@ -1901,9 +1888,9 @@ public void shouldRethrowTaskCorruptedExceptionFromInitialization() {
                 () -> taskManager.checkStateUpdater(time.milliseconds(), noOpResetter)
             );
     
    -        Mockito.verify(tasks).addTask(statefulTask0);
    -        Mockito.verify(tasks).addTask(statefulTask1);
    -        Mockito.verify(stateUpdater).add(statefulTask2);
    +        verify(tasks).addTask(statefulTask0);
    +        verify(tasks).addTask(statefulTask1);
    +        verify(stateUpdater).add(statefulTask2);
             assertEquals(mkSet(taskId00, taskId01), thrown.corruptedTasks());
             assertEquals("Tasks [0_1, 0_0] are corrupted and hence need to be re-initialized", thrown.getMessage());
         }
    @@ -1922,19 +1909,17 @@ public void shouldAddSubscribedTopicsFromAssignmentToTopologyMetadata() {
     
             taskManager.handleAssignment(activeTasksAssignment, standbyTasksAssignment);
     
    -        Mockito.verify(topologyBuilder).addSubscribedTopicsFromAssignment(Mockito.eq(mkSet(t1p1, t1p2, t2p2)), Mockito.anyString());
    -        Mockito.verify(topologyBuilder, never()).addSubscribedTopicsFromAssignment(Mockito.eq(mkSet(t1p3, t1p4)), Mockito.anyString());
    -        Mockito.verify(activeTaskCreator).createTasks(any(), Mockito.eq(activeTasksAssignment));
    +        verify(topologyBuilder).addSubscribedTopicsFromAssignment(eq(mkSet(t1p1, t1p2, t2p2)), anyString());
    +        verify(topologyBuilder, never()).addSubscribedTopicsFromAssignment(eq(mkSet(t1p3, t1p4)), anyString());
    +        verify(activeTaskCreator).createTasks(any(), eq(activeTasksAssignment));
         }
     
         @Test
         public void shouldNotLockAnythingIfStateDirIsEmpty() {
    -        expect(stateDirectory.listNonEmptyTaskDirectories()).andReturn(new ArrayList<>()).once();
    +        when(stateDirectory.listNonEmptyTaskDirectories()).thenReturn(new ArrayList<>());
     
    -        replay(stateDirectory);
             taskManager.handleRebalanceStart(singleton("topic"));
     
    -        verify(stateDirectory);
             assertTrue(taskManager.lockedTaskDirectories().isEmpty());
         }
     
    @@ -1949,10 +1934,8 @@ public void shouldTryToLockValidTaskDirsAtRebalanceStart() throws Exception {
                 taskId10.toString(),
                 "dummy"
             );
    -        replay(stateDirectory);
             taskManager.handleRebalanceStart(singleton("topic"));
     
    -        verify(stateDirectory);
             assertThat(taskManager.lockedTaskDirectories(), is(singleton(taskId01)));
         }
     
    @@ -1960,26 +1943,23 @@ public void shouldTryToLockValidTaskDirsAtRebalanceStart() throws Exception {
         public void shouldUnlockEmptyDirsAtRebalanceStart() throws Exception {
             expectLockObtainedFor(taskId01, taskId10);
             expectDirectoryNotEmpty(taskId01);
    -        expect(stateDirectory.directoryForTaskIsEmpty(taskId10)).andReturn(true);
    -        expectUnlockFor(taskId10);
    +        when(stateDirectory.directoryForTaskIsEmpty(taskId10)).thenReturn(true);
     
             makeTaskFolders(taskId01.toString(), taskId10.toString());
    -        replay(stateDirectory);
             taskManager.handleRebalanceStart(singleton("topic"));
     
    -        verify(stateDirectory);
    +        verify(stateDirectory).unlock(taskId10);
             assertThat(taskManager.lockedTaskDirectories(), is(singleton(taskId01)));
         }
     
         @Test
         public void shouldPauseAllTopicsWithoutStateUpdaterOnRebalanceComplete() {
             final Set assigned = mkSet(t1p0, t1p1);
    -        taskManager.setMainConsumer(mockitoConsumer);
    -        when(mockitoConsumer.assignment()).thenReturn(assigned);
    +        when(consumer.assignment()).thenReturn(assigned);
     
             taskManager.handleRebalanceComplete();
     
    -        Mockito.verify(mockitoConsumer).pause(assigned);
    +        verify(consumer).pause(assigned);
         }
     
         @Test
    @@ -1987,42 +1967,38 @@ public void shouldNotPauseReadyTasksWithStateUpdaterOnRebalanceComplete() {
             final StreamTask statefulTask0 = statefulTask(taskId00, taskId00ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId00Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
    -        taskManager.setMainConsumer(mockitoConsumer);
             when(tasks.allTasks()).thenReturn(mkSet(statefulTask0));
             final Set assigned = mkSet(t1p0, t1p1);
    -        when(mockitoConsumer.assignment()).thenReturn(assigned);
    +        when(consumer.assignment()).thenReturn(assigned);
     
             taskManager.handleRebalanceComplete();
     
    -        Mockito.verify(mockitoConsumer).pause(mkSet(t1p1));
    +        verify(consumer).pause(mkSet(t1p1));
         }
     
         @Test
         public void shouldReleaseLockForUnassignedTasksAfterRebalance() throws Exception {
             expectLockObtainedFor(taskId00, taskId01, taskId02);
             expectDirectoryNotEmpty(taskId00, taskId01, taskId02);
    -        expectUnlockFor(taskId02);
     
             makeTaskFolders(
                 taskId00.toString(),  // active task
                 taskId01.toString(),  // standby task
                 taskId02.toString()   // unassigned but able to lock
             );
    -        replay(stateDirectory);
             taskManager.handleRebalanceStart(singleton("topic"));
     
             assertThat(taskManager.lockedTaskDirectories(), is(mkSet(taskId00, taskId01, taskId02)));
     
             handleAssignment(taskId00Assignment, taskId01Assignment, emptyMap());
    -        reset(consumer);
    -        expectConsumerAssignmentPaused(consumer);
    -        replay(consumer);
     
             taskManager.handleRebalanceComplete();
             assertThat(taskManager.lockedTaskDirectories(), is(mkSet(taskId00, taskId01)));
    -        verify(stateDirectory);
    +
    +        verify(stateDirectory).unlock(taskId02);
    +        verify(consumer).pause(assignment);
         }
     
         @Test
    @@ -2039,31 +2015,28 @@ public void shouldReleaseLockForUnassignedTasksAfterRebalanceWithStateUpdater()
             final StandbyTask unassignedStandbyTask = standbyTask(taskId03, taskId03ChangelogPartitions)
                 .inState(State.CREATED)
                 .withInputPartitions(taskId03Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
    -        taskManager.setMainConsumer(mockitoConsumer);
             when(tasks.allTasksPerId()).thenReturn(mkMap(mkEntry(taskId00, runningStatefulTask)));
             when(stateUpdater.getTasks()).thenReturn(mkSet(standbyTask, restoringStatefulTask));
             when(tasks.allTasks()).thenReturn(mkSet(runningStatefulTask));
             expectLockObtainedFor(taskId00, taskId01, taskId02, taskId03);
             expectDirectoryNotEmpty(taskId00, taskId01, taskId02, taskId03);
    -        expectUnlockFor(taskId03);
             makeTaskFolders(
                 taskId00.toString(),
                 taskId01.toString(),
                 taskId02.toString(),
                 taskId03.toString()
             );
    -        replay(stateDirectory);
     
             final Set assigned = mkSet(t1p0, t1p1, t1p2);
    -        when(mockitoConsumer.assignment()).thenReturn(assigned);
    +        when(consumer.assignment()).thenReturn(assigned);
     
             taskManager.handleRebalanceStart(singleton("topic"));
             taskManager.handleRebalanceComplete();
     
    -        Mockito.verify(mockitoConsumer).pause(mkSet(t1p1, t1p2));
    -        verify(stateDirectory);
    +        verify(consumer).pause(mkSet(t1p1, t1p2));
    +        verify(stateDirectory).unlock(taskId03);
             assertThat(taskManager.lockedTaskDirectories(), is(mkSet(taskId00, taskId01, taskId02)));
         }
     
    @@ -2099,10 +2072,9 @@ public void shouldComputeOffsetSumForRestoringActiveTaskWithStateUpdater() throw
             makeTaskFolders(taskId00.toString());
             final Map changelogOffsetInCheckpoint = mkMap(mkEntry(t1p0changelog, 24L));
             writeCheckpointFile(taskId00, changelogOffsetInCheckpoint);
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(restoringStatefulTask));
    -        replay(stateDirectory);
             taskManager.handleRebalanceStart(singleton("topic"));
     
             assertThat(taskManager.getTaskOffsetSums(), is(mkMap(mkEntry(taskId00, changelogOffset))));
    @@ -2118,10 +2090,9 @@ public void shouldComputeOffsetSumForRestoringStandbyTaskWithStateUpdater() thro
             makeTaskFolders(taskId00.toString());
             final Map changelogOffsetInCheckpoint = mkMap(mkEntry(t1p0changelog, 24L));
             writeCheckpointFile(taskId00, changelogOffsetInCheckpoint);
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(stateUpdater.getTasks()).thenReturn(mkSet(restoringStandbyTask));
    -        replay(stateDirectory);
             taskManager.handleRebalanceStart(singleton("topic"));
     
             assertThat(taskManager.getTaskOffsetSums(), is(mkMap(mkEntry(taskId00, changelogOffset))));
    @@ -2144,7 +2115,7 @@ public void shouldComputeOffsetSumForRunningStatefulTaskAndRestoringTaskWithStat
                 .thenReturn(mkMap(mkEntry(t1p1changelog, changelogOffsetOfRestoringStatefulTask)));
             when(restoringStandbyTask.changelogOffsets())
                 .thenReturn(mkMap(mkEntry(t1p2changelog, changelogOffsetOfRestoringStandbyTask)));
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
             when(tasks.allTasksPerId()).thenReturn(mkMap(mkEntry(taskId00, runningStatefulTask)));
             when(stateUpdater.getTasks()).thenReturn(mkSet(restoringStandbyTask, restoringStatefulTask));
    @@ -2175,7 +2146,6 @@ private void computeOffsetSumAndVerify(final Map changelog
             expectLockObtainedFor(taskId00);
             expectDirectoryNotEmpty(taskId00);
             makeTaskFolders(taskId00.toString());
    -        replay(stateDirectory);
     
             taskManager.handleRebalanceStart(singleton("topic"));
             final StateMachineTask restoringTask = handleAssignment(
    @@ -2199,7 +2169,6 @@ public void shouldComputeOffsetSumForStandbyTask() throws Exception {
             expectLockObtainedFor(taskId00);
             expectDirectoryNotEmpty(taskId00);
             makeTaskFolders(taskId00.toString());
    -        replay(stateDirectory);
     
             taskManager.handleRebalanceStart(singleton("topic"));
             final StateMachineTask restoringTask = handleAssignment(
    @@ -2224,7 +2193,6 @@ public void shouldComputeOffsetSumForUnassignedTaskWeCanLock() throws Exception
             makeTaskFolders(taskId00.toString());
             writeCheckpointFile(taskId00, changelogOffsets);
     
    -        replay(stateDirectory);
             taskManager.handleRebalanceStart(singleton("topic"));
     
             assertThat(taskManager.getTaskOffsetSums(), is(expectedOffsetSums));
    @@ -2241,11 +2209,10 @@ public void shouldComputeOffsetSumFromCheckpointFileForUninitializedTask() throw
             expectLockObtainedFor(taskId00);
             makeTaskFolders(taskId00.toString());
             writeCheckpointFile(taskId00, changelogOffsets);
    -        replay(stateDirectory);
     
             taskManager.handleRebalanceStart(singleton("topic"));
             final StateMachineTask uninitializedTask = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singleton(uninitializedTask));
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singleton(uninitializedTask));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
     
    @@ -2265,13 +2232,12 @@ public void shouldComputeOffsetSumFromCheckpointFileForClosedTask() throws Excep
             expectLockObtainedFor(taskId00);
             makeTaskFolders(taskId00.toString());
             writeCheckpointFile(taskId00, changelogOffsets);
    -        replay(stateDirectory);
     
             final StateMachineTask closedTask = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
     
             taskManager.handleRebalanceStart(singleton("topic"));
     
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singleton(closedTask));
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singleton(closedTask));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
     
    @@ -2286,7 +2252,6 @@ public void shouldComputeOffsetSumFromCheckpointFileForClosedTask() throws Excep
         public void shouldNotReportOffsetSumsForTaskWeCantLock() throws Exception {
             expectLockFailedFor(taskId00);
             makeTaskFolders(taskId00.toString());
    -        replay(stateDirectory);
             taskManager.handleRebalanceStart(singleton("topic"));
             assertTrue(taskManager.lockedTaskDirectories().isEmpty());
     
    @@ -2298,12 +2263,10 @@ public void shouldNotReportOffsetSumsAndReleaseLockForUnassignedTaskWithoutCheck
             expectLockObtainedFor(taskId00);
             makeTaskFolders(taskId00.toString());
             expectDirectoryNotEmpty(taskId00);
    -        expect(stateDirectory.checkpointFileFor(taskId00)).andReturn(getCheckpointFile(taskId00));
    -        replay(stateDirectory);
    +        when(stateDirectory.checkpointFileFor(taskId00)).thenReturn(getCheckpointFile(taskId00));
             taskManager.handleRebalanceStart(singleton("topic"));
     
             assertTrue(taskManager.getTaskOffsetSums().isEmpty());
    -        verify(stateDirectory);
         }
     
         @Test
    @@ -2319,7 +2282,6 @@ public void shouldPinOffsetSumToLongMaxValueInCaseOfOverflow() throws Exception
             expectLockObtainedFor(taskId00);
             makeTaskFolders(taskId00.toString());
             writeCheckpointFile(taskId00, changelogOffsets);
    -        replay(stateDirectory);
             taskManager.handleRebalanceStart(singleton("topic"));
     
             assertThat(taskManager.getTaskOffsetSums(), is(expectedOffsetSums));
    @@ -2332,19 +2294,9 @@ public void shouldCloseActiveUnassignedSuspendedTasksWhenClosingRevokedTasks() {
             task00.setCommittableOffsetsAndMetadata(offsets);
     
             // first `handleAssignment`
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        expectLastCall();
    -
    -        // `handleRevocation`
    -        consumer.commitSync(offsets);
    -        expectLastCall();
    +        when(consumer.assignment()).thenReturn(assignment);
     
    -        // second `handleAssignment`
    -        consumer.commitSync(offsets);
    -        expectLastCall();
    -
    -        replay(consumer);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -2357,7 +2309,7 @@ public void shouldCloseActiveUnassignedSuspendedTasksWhenClosingRevokedTasks() {
             assertThat(task00.state(), is(Task.State.CLOSED));
             assertThat(taskManager.activeTaskMap(), Matchers.anEmptyMap());
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
         }
     
         @Test
    @@ -2369,12 +2321,7 @@ public void closeClean() {
                 }
             };
     
    -        // first `handleAssignment`
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        expectLastCall();
    -
    -        replay(consumer);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             taskManager.handleRevocation(taskId00Partitions);
    @@ -2390,7 +2337,7 @@ public void closeClean() {
                 is("Encounter unexpected fatal error for task 0_0")
             );
             assertThat(thrown.getCause().getMessage(), is("KABOOM!"));
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
         }
     
         @Test
    @@ -2399,24 +2346,24 @@ public void shouldCloseActiveTasksWhenHandlingLostTasks() throws Exception {
             final StateMachineTask task01 = new StateMachineTask(taskId01, taskId01Partitions, false, stateManager);
     
             // `handleAssignment`
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
             when(standbyTaskCreator.createTasks(taskId01Assignment)).thenReturn(singletonList(task01));
     
    -        makeTaskFolders(taskId00.toString(), taskId01.toString());
    +        final ArrayList taskFolders = new ArrayList<>(2);
    +        taskFolders.add(new TaskDirectory(testFolder.newFolder(taskId00.toString()), null));
    +        taskFolders.add(new TaskDirectory(testFolder.newFolder(taskId01.toString()), null));
    +
    +        when(stateDirectory.listNonEmptyTaskDirectories())
    +            .thenReturn(taskFolders)
    +            .thenReturn(new ArrayList<>());
    +
             expectLockObtainedFor(taskId00, taskId01);
             expectDirectoryNotEmpty(taskId00, taskId01);
     
    -        // The second attempt will return empty tasks.
    -        makeTaskFolders();
    -        expectLockObtainedFor();
    -        replay(stateDirectory);
    -
             taskManager.handleRebalanceStart(emptySet());
             assertThat(taskManager.lockedTaskDirectories(), Matchers.is(mkSet(taskId00, taskId01)));
     
    -        replay(consumer);
    -
             taskManager.handleAssignment(taskId00Assignment, taskId01Assignment);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
             assertThat(task00.state(), is(Task.State.RUNNING));
    @@ -2436,7 +2383,7 @@ public void shouldCloseActiveTasksWhenHandlingLostTasks() throws Exception {
             taskManager.handleRebalanceStart(emptySet());
     
             assertThat(taskManager.lockedTaskDirectories(), is(emptySet()));
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
         }
     
         @Test
    @@ -2445,7 +2392,7 @@ public void shouldReInitializeThreadProducerOnHandleLostAllIfEosV2Enabled() {
     
             taskManager.handleLostAll();
     
    -        Mockito.verify(activeTaskCreator).reInitializeThreadProducer();
    +        verify(activeTaskCreator).reInitializeThreadProducer();
         }
     
         @Test
    @@ -2455,16 +2402,12 @@ public void shouldThrowWhenHandlingClosingTasksOnProducerCloseError() {
             task00.setCommittableOffsetsAndMetadata(offsets);
     
             // `handleAssignment`
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             // `handleAssignment`
    -        consumer.commitSync(offsets);
    -        expectLastCall();
             doThrow(new RuntimeException("KABOOM!")).when(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
     
    -        replay(consumer);
    -
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
             assertThat(task00.state(), is(Task.State.RUNNING));
    @@ -2492,9 +2435,8 @@ public void shouldReAddRevivedTasksToStateUpdater() {
             final StandbyTask corruptedStandbyTask = standbyTask(taskId02, taskId02ChangelogPartitions)
                 .inState(State.RUNNING)
                 .withInputPartitions(taskId02Partitions).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
    -        taskManager.setMainConsumer(mockitoConsumer);
             when(tasks.task(taskId03)).thenReturn(corruptedActiveTask);
             when(tasks.task(taskId02)).thenReturn(corruptedStandbyTask);
     
    @@ -2506,16 +2448,16 @@ public void shouldReAddRevivedTasksToStateUpdater() {
             final InOrder standbyTaskOrder = inOrder(corruptedStandbyTask);
             standbyTaskOrder.verify(corruptedStandbyTask).closeDirty();
             standbyTaskOrder.verify(corruptedStandbyTask).revive();
    -        Mockito.verify(tasks).removeTask(corruptedActiveTask);
    -        Mockito.verify(tasks).removeTask(corruptedStandbyTask);
    -        Mockito.verify(tasks).addPendingTasksToInit(mkSet(corruptedActiveTask));
    -        Mockito.verify(tasks).addPendingTasksToInit(mkSet(corruptedStandbyTask));
    -        Mockito.verify(mockitoConsumer).assignment();
    +        verify(tasks).removeTask(corruptedActiveTask);
    +        verify(tasks).removeTask(corruptedStandbyTask);
    +        verify(tasks).addPendingTasksToInit(mkSet(corruptedActiveTask));
    +        verify(tasks).addPendingTasksToInit(mkSet(corruptedStandbyTask));
    +        verify(consumer).assignment();
         }
     
         @Test
         public void shouldReviveCorruptTasks() {
    -        final ProcessorStateManager stateManager = Mockito.mock(ProcessorStateManager.class);
    +        final ProcessorStateManager stateManager = mock(ProcessorStateManager.class);
     
             final AtomicBoolean enforcedCheckpoint = new AtomicBoolean(false);
             final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager) {
    @@ -2529,10 +2471,10 @@ public void postCommit(final boolean enforceCheckpoint) {
             };
     
             // `handleAssignment`
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        expect(consumer.assignment()).andReturn(taskId00Partitions);
    -        replay(consumer);
    +        when(consumer.assignment())
    +            .thenReturn(assignment)
    +            .thenReturn(taskId00Partitions);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), tp -> assertThat(tp, is(empty()))), is(true));
    @@ -2548,13 +2490,12 @@ public void postCommit(final boolean enforceCheckpoint) {
             assertThat(taskManager.activeTaskMap(), is(singletonMap(taskId00, task00)));
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
     
    -        verify(consumer);
    -        Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00Partitions);
    +        verify(stateManager).markChangelogAsCorrupted(taskId00Partitions);
         }
     
         @Test
         public void shouldReviveCorruptTasksEvenIfTheyCannotCloseClean() {
    -        final ProcessorStateManager stateManager = Mockito.mock(ProcessorStateManager.class);
    +        final ProcessorStateManager stateManager = mock(ProcessorStateManager.class);
     
             final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager) {
                 @Override
    @@ -2564,10 +2505,10 @@ public void suspend() {
                 }
             };
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        expect(consumer.assignment()).andReturn(taskId00Partitions);
    -        replay(consumer);
    +        when(consumer.assignment())
    +            .thenReturn(assignment)
    +            .thenReturn(taskId00Partitions);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), tp -> assertThat(tp, is(empty()))), is(true));
    @@ -2581,31 +2522,28 @@ public void suspend() {
             assertThat(taskManager.activeTaskMap(), is(singletonMap(taskId00, task00)));
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
     
    -        verify(consumer);
    -        Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00Partitions);
    +        verify(stateManager).markChangelogAsCorrupted(taskId00Partitions);
         }
     
         @Test
         public void shouldCommitNonCorruptedTasksOnTaskCorruptedException() {
    -        final ProcessorStateManager stateManager = Mockito.mock(ProcessorStateManager.class);
    +        final ProcessorStateManager stateManager = mock(ProcessorStateManager.class);
     
             final StateMachineTask corruptedTask = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
             final StateMachineTask nonCorruptedTask = new StateMachineTask(taskId01, taskId01Partitions, true, stateManager);
     
    -        final Map> assignment = new HashMap<>(taskId00Assignment);
    -        assignment.putAll(taskId01Assignment);
    +        final Map> firstAssignment = new HashMap<>(taskId00Assignment);
    +        firstAssignment.putAll(taskId01Assignment);
     
             // `handleAssignment`
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment)))
    +        when(activeTaskCreator.createTasks(any(), eq(firstAssignment)))
                 .thenReturn(asList(corruptedTask, nonCorruptedTask));
    -        expectRestoreToBeCompleted(consumer);
    -        expect(consumer.assignment()).andReturn(taskId00Partitions);
    -        // check that we should not commit empty map either
    -        consumer.commitSync(eq(emptyMap()));
    -        expectLastCall().andStubThrow(new AssertionError("should not invoke commitSync when offset map is empty"));
    -        replay(consumer);
     
    -        taskManager.handleAssignment(assignment, emptyMap());
    +        when(consumer.assignment())
    +            .thenReturn(assignment)
    +            .thenReturn(taskId00Partitions);
    +
    +        taskManager.handleAssignment(firstAssignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), tp -> assertThat(tp, is(empty()))), is(true));
     
             assertThat(nonCorruptedTask.state(), is(Task.State.RUNNING));
    @@ -2618,13 +2556,14 @@ public void shouldCommitNonCorruptedTasksOnTaskCorruptedException() {
             assertThat(nonCorruptedTask.partitionsForOffsetReset, equalTo(Collections.emptySet()));
             assertThat(corruptedTask.partitionsForOffsetReset, equalTo(taskId00Partitions));
     
    -        verify(consumer);
    -        Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00Partitions);
    +        // check that we should not commit empty map either
    +        verify(consumer, never()).commitSync(emptyMap());
    +        verify(stateManager).markChangelogAsCorrupted(taskId00Partitions);
         }
     
         @Test
         public void shouldNotCommitNonRunningNonCorruptedTasks() {
    -        final ProcessorStateManager stateManager = Mockito.mock(ProcessorStateManager.class);
    +        final ProcessorStateManager stateManager = mock(ProcessorStateManager.class);
     
             final StateMachineTask corruptedTask = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
             final StateMachineTask nonRunningNonCorruptedTask = new StateMachineTask(taskId01, taskId01Partitions, true, stateManager);
    @@ -2635,10 +2574,9 @@ public void shouldNotCommitNonRunningNonCorruptedTasks() {
             assignment.putAll(taskId01Assignment);
     
             // `handleAssignment`
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment)))
    +        when(activeTaskCreator.createTasks(any(), eq(assignment)))
                 .thenReturn(asList(corruptedTask, nonRunningNonCorruptedTask));
    -        expect(consumer.assignment()).andReturn(taskId00Partitions);
    -        replay(consumer);
    +        when(consumer.assignment()).thenReturn(taskId00Partitions);
     
             taskManager.handleAssignment(assignment, emptyMap());
     
    @@ -2650,8 +2588,7 @@ public void shouldNotCommitNonRunningNonCorruptedTasks() {
             assertThat(corruptedTask.partitionsForOffsetReset, equalTo(taskId00Partitions));
     
             assertFalse(nonRunningNonCorruptedTask.commitPrepared);
    -        verify(consumer);
    -        Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00Partitions);
    +        verify(stateManager).markChangelogAsCorrupted(taskId00Partitions);
         }
     
         @Test
    @@ -2665,22 +2602,20 @@ public void shouldNotCommitNonCorruptedRestoringActiveTasksAndNotCommitRunningSt
             final StreamTask corruptedTask = statefulTask(taskId02, taskId02ChangelogPartitions)
                 .withInputPartitions(taskId02Partitions)
                 .inState(State.RUNNING).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             when(tasks.allTasksPerId()).thenReturn(mkMap(mkEntry(taskId02, corruptedTask)));
             when(tasks.task(taskId02)).thenReturn(corruptedTask);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
    -        taskManager.setMainConsumer(mockitoConsumer);
    -        when(stateUpdater.getTasks()).thenReturn(mkSet(activeRestoringTask, standbyTask));
    -        when(mockitoConsumer.assignment()).thenReturn(intersection(HashSet::new, taskId00Partitions, taskId01Partitions, taskId02Partitions));
    +        when(consumer.assignment()).thenReturn(intersection(HashSet::new, taskId00Partitions, taskId01Partitions, taskId02Partitions));
     
             taskManager.handleCorruption(mkSet(taskId02));
     
    -        Mockito.verify(activeRestoringTask, never()).commitNeeded();
    -        Mockito.verify(activeRestoringTask, never()).prepareCommit();
    -        Mockito.verify(activeRestoringTask, never()).postCommit(Mockito.anyBoolean());
    -        Mockito.verify(standbyTask, never()).commitNeeded();
    -        Mockito.verify(standbyTask, never()).prepareCommit();
    -        Mockito.verify(standbyTask, never()).postCommit(Mockito.anyBoolean());
    +        verify(activeRestoringTask, never()).commitNeeded();
    +        verify(activeRestoringTask, never()).prepareCommit();
    +        verify(activeRestoringTask, never()).postCommit(anyBoolean());
    +        verify(standbyTask, never()).commitNeeded();
    +        verify(standbyTask, never()).prepareCommit();
    +        verify(standbyTask, never()).postCommit(anyBoolean());
         }
     
         @Test
    @@ -2695,7 +2630,7 @@ public void shouldNotCommitNonCorruptedRestoringActiveTasksAndCommitRunningStand
             final StreamTask corruptedTask = statefulTask(taskId02, taskId02ChangelogPartitions)
                 .withInputPartitions(taskId02Partitions)
                 .inState(State.RUNNING).build();
    -        final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
    +        final TasksRegistry tasks = mock(TasksRegistry.class);
             when(tasks.allTasksPerId()).thenReturn(mkMap(
                 mkEntry(taskId00, activeRestoringTask),
                 mkEntry(taskId01, standbyTask),
    @@ -2703,21 +2638,20 @@ public void shouldNotCommitNonCorruptedRestoringActiveTasksAndCommitRunningStand
             ));
             when(tasks.task(taskId02)).thenReturn(corruptedTask);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, false);
    -        taskManager.setMainConsumer(mockitoConsumer);
    -        when(mockitoConsumer.assignment()).thenReturn(intersection(HashSet::new, taskId00Partitions, taskId01Partitions, taskId02Partitions));
    +        when(consumer.assignment()).thenReturn(intersection(HashSet::new, taskId00Partitions, taskId01Partitions, taskId02Partitions));
     
             taskManager.handleCorruption(mkSet(taskId02));
     
    -        Mockito.verify(activeRestoringTask, never()).commitNeeded();
    -        Mockito.verify(activeRestoringTask, never()).prepareCommit();
    -        Mockito.verify(activeRestoringTask, never()).postCommit(Mockito.anyBoolean());
    -        Mockito.verify(standbyTask).prepareCommit();
    -        Mockito.verify(standbyTask).postCommit(Mockito.anyBoolean());
    +        verify(activeRestoringTask, never()).commitNeeded();
    +        verify(activeRestoringTask, never()).prepareCommit();
    +        verify(activeRestoringTask, never()).postCommit(anyBoolean());
    +        verify(standbyTask).prepareCommit();
    +        verify(standbyTask).postCommit(anyBoolean());
         }
     
         @Test
         public void shouldCleanAndReviveCorruptedStandbyTasksBeforeCommittingNonCorruptedTasks() {
    -        final ProcessorStateManager stateManager = Mockito.mock(ProcessorStateManager.class);
    +        final ProcessorStateManager stateManager = mock(ProcessorStateManager.class);
     
             final StateMachineTask corruptedStandby = new StateMachineTask(taskId00, taskId00Partitions, false, stateManager);
             final StateMachineTask runningNonCorruptedActive = new StateMachineTask(taskId01, taskId01Partitions, true, stateManager) {
    @@ -2728,13 +2662,11 @@ public Map prepareCommit() {
             };
     
             // handleAssignment
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId01Assignment)))
    +        when(activeTaskCreator.createTasks(any(), eq(taskId01Assignment)))
                 .thenReturn(singleton(runningNonCorruptedActive));
             when(standbyTaskCreator.createTasks(taskId00Assignment)).thenReturn(singleton(corruptedStandby));
     
    -        expectRestoreToBeCompleted(consumer);
    -
    -        replay(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
     
             taskManager.handleAssignment(taskId01Assignment, taskId00Assignment);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -2751,14 +2683,13 @@ public Map prepareCommit() {
     
             assertThat(corruptedStandby.commitPrepared, is(true));
             assertThat(corruptedStandby.state(), is(Task.State.CREATED));
    -        verify(consumer);
    -        Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00Partitions);
    +        verify(stateManager).markChangelogAsCorrupted(taskId00Partitions);
         }
     
         @Test
         public void shouldNotAttemptToCommitInHandleCorruptedDuringARebalance() {
    -        final ProcessorStateManager stateManager = Mockito.mock(ProcessorStateManager.class);
    -        expect(stateDirectory.listNonEmptyTaskDirectories()).andStubReturn(new ArrayList<>());
    +        final ProcessorStateManager stateManager = mock(ProcessorStateManager.class);
    +        when(stateDirectory.listNonEmptyTaskDirectories()).thenReturn(new ArrayList<>());
     
             final StateMachineTask corruptedActive = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
     
    @@ -2768,21 +2699,19 @@ public void shouldNotAttemptToCommitInHandleCorruptedDuringARebalance() {
             uncorruptedActive.setCommitNeeded();
     
             // handleAssignment
    -        final Map> assignment = new HashMap<>();
    -        assignment.putAll(taskId00Assignment);
    -        assignment.putAll(taskId01Assignment);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment)))
    +        final Map> firstAssignement = new HashMap<>();
    +        firstAssignement.putAll(taskId00Assignment);
    +        firstAssignement.putAll(taskId01Assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(firstAssignement)))
                 .thenReturn(asList(corruptedActive, uncorruptedActive));
     
    -        expectRestoreToBeCompleted(consumer);
    -
    -        expect(consumer.assignment()).andStubReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions));
    -
    -        replay(consumer, stateDirectory);
    +        when(consumer.assignment())
    +            .thenReturn(assignment)
    +            .thenReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions));
     
             uncorruptedActive.setCommittableOffsetsAndMetadata(offsets);
     
    -        taskManager.handleAssignment(assignment, emptyMap());
    +        taskManager.handleAssignment(firstAssignement, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
     
             assertThat(uncorruptedActive.state(), is(Task.State.RUNNING));
    @@ -2800,12 +2729,11 @@ public void shouldNotAttemptToCommitInHandleCorruptedDuringARebalance() {
             assertThat(uncorruptedActive.commitCompleted, is(false));
     
             assertThat(uncorruptedActive.state(), is(State.RUNNING));
    -        verify(consumer);
         }
     
         @Test
         public void shouldCloseAndReviveUncorruptedTasksWhenTimeoutExceptionThrownFromCommitWithALOS() {
    -        final ProcessorStateManager stateManager = Mockito.mock(ProcessorStateManager.class);
    +        final ProcessorStateManager stateManager = mock(ProcessorStateManager.class);
     
             final StateMachineTask corruptedActive = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
             final StateMachineTask uncorruptedActive = new StateMachineTask(taskId01, taskId01Partitions, true, stateManager) {
    @@ -2818,22 +2746,19 @@ public void markChangelogAsCorrupted(final Collection partitions
             uncorruptedActive.setCommittableOffsetsAndMetadata(offsets);
     
             // handleAssignment
    -        final Map> assignment = new HashMap<>();
    -        assignment.putAll(taskId00Assignment);
    -        assignment.putAll(taskId01Assignment);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment)))
    +        final Map> firstAssignment = new HashMap<>();
    +        firstAssignment.putAll(taskId00Assignment);
    +        firstAssignment.putAll(taskId01Assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(firstAssignment)))
                 .thenReturn(asList(corruptedActive, uncorruptedActive));
     
    -        expectRestoreToBeCompleted(consumer);
    -
    -        consumer.commitSync(offsets);
    -        expectLastCall().andThrow(new TimeoutException());
    +        when(consumer.assignment())
    +            .thenReturn(assignment)
    +            .thenReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions));
     
    -        expect(consumer.assignment()).andStubReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions));
    +        doThrow(new TimeoutException()).when(consumer).commitSync(offsets);
     
    -        replay(consumer);
    -
    -        taskManager.handleAssignment(assignment, emptyMap());
    +        taskManager.handleAssignment(firstAssignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
     
             assertThat(uncorruptedActive.state(), is(Task.State.RUNNING));
    @@ -2861,8 +2786,7 @@ public void markChangelogAsCorrupted(final Collection partitions
     
             assertThat(corruptedActive.state(), is(Task.State.CREATED));
             assertThat(uncorruptedActive.state(), is(Task.State.CREATED));
    -        verify(consumer);
    -        Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00Partitions);
    +        verify(stateManager).markChangelogAsCorrupted(taskId00Partitions);
         }
     
         @Test
    @@ -2870,7 +2794,7 @@ public void shouldCloseAndReviveUncorruptedTasksWhenTimeoutExceptionThrownFromCo
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.EXACTLY_ONCE_V2, false);
             final StreamsProducer producer = mock(StreamsProducer.class);
             when(activeTaskCreator.threadProducer()).thenReturn(producer);
    -        final ProcessorStateManager stateManager = Mockito.mock(ProcessorStateManager.class);
    +        final ProcessorStateManager stateManager = mock(ProcessorStateManager.class);
     
             final AtomicBoolean corruptedTaskChangelogMarkedAsCorrupted = new AtomicBoolean(false);
             final StateMachineTask corruptedActiveTask = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager) {
    @@ -2893,24 +2817,22 @@ public void markChangelogAsCorrupted(final Collection partitions
             uncorruptedActiveTask.setCommittableOffsetsAndMetadata(offsets);
     
             // handleAssignment
    -        final Map> assignment = new HashMap<>();
    -        assignment.putAll(taskId00Assignment);
    -        assignment.putAll(taskId01Assignment);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment)))
    +        final Map> firstAssignment = new HashMap<>();
    +        firstAssignment.putAll(taskId00Assignment);
    +        firstAssignment.putAll(taskId01Assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(firstAssignment)))
                 .thenReturn(asList(corruptedActiveTask, uncorruptedActiveTask));
     
    -        expectRestoreToBeCompleted(consumer);
    +        when(consumer.assignment())
    +            .thenReturn(assignment)
    +            .thenReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions));
     
             final ConsumerGroupMetadata groupMetadata = new ConsumerGroupMetadata("appId");
    -        expect(consumer.groupMetadata()).andReturn(groupMetadata);
    +        when(consumer.groupMetadata()).thenReturn(groupMetadata);
     
             doThrow(new TimeoutException()).when(producer).commitTransaction(offsets, groupMetadata);
     
    -        expect(consumer.assignment()).andStubReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions));
    -
    -        replay(consumer);
    -
    -        taskManager.handleAssignment(assignment, emptyMap());
    +        taskManager.handleAssignment(firstAssignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
     
             assertThat(uncorruptedActiveTask.state(), is(Task.State.RUNNING));
    @@ -2944,9 +2866,8 @@ public void markChangelogAsCorrupted(final Collection partitions
             assertThat(uncorruptedActiveTask.state(), is(Task.State.CREATED));
             assertThat(corruptedTaskChangelogMarkedAsCorrupted.get(), is(true));
             assertThat(uncorruptedTaskChangelogMarkedAsCorrupted.get(), is(true));
    -        verify(consumer);
    -        Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00ChangelogPartitions);
    -        Mockito.verify(stateManager).markChangelogAsCorrupted(taskId01ChangelogPartitions);
    +        verify(stateManager).markChangelogAsCorrupted(taskId00ChangelogPartitions);
    +        verify(stateManager).markChangelogAsCorrupted(taskId01ChangelogPartitions);
         }
     
         @Test
    @@ -2978,16 +2899,14 @@ public void markChangelogAsCorrupted(final Collection partitions
                 mkEntry(taskId02, taskId02Partitions)
             );
     
    -        expectRestoreToBeCompleted(consumer);
    +        when(consumer.assignment())
    +            .thenReturn(assignment)
    +            .thenReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions, taskId02Partitions));
     
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive)))
    +        when(activeTaskCreator.createTasks(any(), eq(assignmentActive)))
                 .thenReturn(asList(revokedActiveTask, unrevokedActiveTaskWithCommitNeeded, unrevokedActiveTaskWithoutCommitNeeded));
    -        expectLastCall();
    -        consumer.commitSync(expectedCommittedOffsets);
    -        expectLastCall().andThrow(new TimeoutException());
    -        expect(consumer.assignment()).andStubReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions, taskId02Partitions));
     
    -        replay(consumer);
    +        doThrow(new TimeoutException()).when(consumer).commitSync(expectedCommittedOffsets);
     
             taskManager.handleAssignment(assignmentActive, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -3007,7 +2926,7 @@ public void shouldCloseAndReviveUncorruptedTasksWhenTimeoutExceptionThrownFromCo
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.EXACTLY_ONCE_V2, false);
             final StreamsProducer producer = mock(StreamsProducer.class);
             when(activeTaskCreator.threadProducer()).thenReturn(producer);
    -        final ProcessorStateManager stateManager = Mockito.mock(ProcessorStateManager.class);
    +        final ProcessorStateManager stateManager = mock(ProcessorStateManager.class);
     
             final StateMachineTask revokedActiveTask = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
             final Map revokedActiveTaskOffsets = singletonMap(t1p0, new OffsetAndMetadata(0L, null));
    @@ -3038,20 +2957,18 @@ public void markChangelogAsCorrupted(final Collection partitions
                 mkEntry(taskId02, taskId02Partitions)
                 );
     
    -        expectRestoreToBeCompleted(consumer);
    +        when(consumer.assignment())
    +            .thenReturn(assignment)
    +            .thenReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions, taskId02Partitions));
     
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive)))
    +        when(activeTaskCreator.createTasks(any(), eq(assignmentActive)))
                 .thenReturn(asList(revokedActiveTask, unrevokedActiveTask, unrevokedActiveTaskWithoutCommitNeeded));
     
             final ConsumerGroupMetadata groupMetadata = new ConsumerGroupMetadata("appId");
    -        expect(consumer.groupMetadata()).andReturn(groupMetadata);
    +        when(consumer.groupMetadata()).thenReturn(groupMetadata);
     
             doThrow(new TimeoutException()).when(producer).commitTransaction(expectedCommittedOffsets, groupMetadata);
     
    -        expect(consumer.assignment()).andStubReturn(union(HashSet::new, taskId00Partitions, taskId01Partitions, taskId02Partitions));
    -
    -        replay(consumer);
    -
             taskManager.handleAssignment(assignmentActive, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
             assertThat(revokedActiveTask.state(), is(Task.State.RUNNING));
    @@ -3069,19 +2986,16 @@ public void markChangelogAsCorrupted(final Collection partitions
             assertThat(revokedActiveTask.state(), is(State.SUSPENDED));
             assertThat(unrevokedActiveTask.state(), is(State.CREATED));
             assertThat(unrevokedActiveTaskWithoutCommitNeeded.state(), is(State.RUNNING));
    -        Mockito.verify(stateManager).markChangelogAsCorrupted(taskId00ChangelogPartitions);
    -        Mockito.verify(stateManager).markChangelogAsCorrupted(taskId01ChangelogPartitions);
    +        verify(stateManager).markChangelogAsCorrupted(taskId00ChangelogPartitions);
    +        verify(stateManager).markChangelogAsCorrupted(taskId01ChangelogPartitions);
         }
     
         @Test
         public void shouldCloseStandbyUnassignedTasksWhenCreatingNewTasks() {
             final Task task00 = new StateMachineTask(taskId00, taskId00Partitions, false, stateManager);
     
    -        expectRestoreToBeCompleted(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
             when(standbyTaskCreator.createTasks(taskId00Assignment)).thenReturn(singletonList(task00));
    -        consumer.commitSync(Collections.emptyMap());
    -        expectLastCall();
    -        replay(consumer);
     
             taskManager.handleAssignment(emptyMap(), taskId00Assignment);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -3098,12 +3012,9 @@ public void shouldAddNonResumedSuspendedTasks() {
             final Task task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
             final Task task01 = new StateMachineTask(taskId01, taskId01Partitions, false, stateManager);
     
    -        expectRestoreToBeCompleted(consumer);
    -        // expect these calls twice (because we're going to tryToCompleteRestoration twice)
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
             when(standbyTaskCreator.createTasks(taskId01Assignment)).thenReturn(singletonList(task01));
    -        replay(consumer);
     
             taskManager.handleAssignment(taskId00Assignment, taskId01Assignment);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -3115,18 +3026,18 @@ public void shouldAddNonResumedSuspendedTasks() {
             assertThat(task00.state(), is(Task.State.RUNNING));
             assertThat(task01.state(), is(Task.State.RUNNING));
     
    -        Mockito.verify(activeTaskCreator).createTasks(any(), Mockito.eq(emptyMap()));
    +        // expect these calls twice (because we're going to tryToCompleteRestoration twice)
    +        verify(activeTaskCreator).createTasks(any(), eq(emptyMap()));
    +        verify(consumer, times(2)).assignment();
    +        verify(consumer, times(2)).resume(assignment);
         }
     
         @Test
         public void shouldUpdateInputPartitionsAfterRebalance() {
             final Task task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
     
    -        expectRestoreToBeCompleted(consumer);
    -        // expect these calls twice (because we're going to tryToCompleteRestoration twice)
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        replay(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -3138,8 +3049,10 @@ public void shouldUpdateInputPartitionsAfterRebalance() {
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
             assertThat(task00.state(), is(Task.State.RUNNING));
             assertEquals(newPartitionsSet, task00.inputPartitions());
    -        verify(consumer);
    -        Mockito.verify(activeTaskCreator).createTasks(any(), Mockito.eq(emptyMap()));
    +        // expect these calls twice (because we're going to tryToCompleteRestoration twice)
    +        verify(consumer, times(2)).resume(assignment);
    +        verify(consumer, times(2)).assignment();
    +        verify(activeTaskCreator).createTasks(any(), eq(emptyMap()));
         }
     
         @Test
    @@ -3147,9 +3060,7 @@ public void shouldAddNewActiveTasks() {
             final Map> assignment = taskId00Assignment;
             final Task task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
     
    -        taskManager.setMainConsumer(mockitoConsumer);
    -
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment))).thenReturn(singletonList(task00));
    +        when(activeTaskCreator.createTasks(any(), eq(assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(assignment, emptyMap());
     
    @@ -3160,9 +3071,9 @@ public void shouldAddNewActiveTasks() {
             assertThat(task00.state(), is(Task.State.RUNNING));
             assertThat(taskManager.activeTaskMap(), Matchers.equalTo(singletonMap(taskId00, task00)));
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
    -        Mockito.verify(changeLogReader).enforceRestoreActive();
    -        Mockito.verify(mockitoConsumer).assignment();
    -        Mockito.verify(mockitoConsumer).resume(Mockito.eq(emptySet()));
    +        verify(changeLogReader).enforceRestoreActive();
    +        verify(consumer).assignment();
    +        verify(consumer).resume(eq(emptySet()));
         }
     
         @Test
    @@ -3184,9 +3095,7 @@ public void initializeIfNeeded() {
                 }
             };
     
    -        taskManager.setMainConsumer(mockitoConsumer);
    -
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment))).thenReturn(asList(task00, task01));
    +        when(activeTaskCreator.createTasks(any(), eq(assignment))).thenReturn(asList(task00, task01));
     
             taskManager.handleAssignment(assignment, emptyMap());
     
    @@ -3202,8 +3111,8 @@ public void initializeIfNeeded() {
                 Matchers.equalTo(mkMap(mkEntry(taskId00, task00), mkEntry(taskId01, task01)))
             );
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
    -        Mockito.verify(changeLogReader).enforceRestoreActive();
    -        Mockito.verifyNoInteractions(mockitoConsumer);
    +        verify(changeLogReader).enforceRestoreActive();
    +        verifyNoInteractions(consumer);
         }
     
         @Test
    @@ -3218,9 +3127,7 @@ public void completeRestoration(final java.util.function.Consumer offsets = singletonMap(t1p0, new OffsetAndMetadata(0L, null));
             task00.setCommittableOffsetsAndMetadata(offsets);
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        consumer.commitSync(offsets);
    -        expectLastCall();
    -
    -        replay(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -3293,9 +3196,9 @@ public void shouldCommitAllActiveTasksThatNeedCommittingOnHandleRevocationWithEo
             final Map> assignmentStandby = mkMap(
                 mkEntry(taskId10, taskId10Partitions)
             );
    -        expectRestoreToBeCompleted(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
     
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive)))
    +        when(activeTaskCreator.createTasks(any(), eq(assignmentActive)))
                 .thenReturn(asList(task00, task01, task02));
     
             when(activeTaskCreator.threadProducer()).thenReturn(producer);
    @@ -3303,20 +3206,12 @@ public void shouldCommitAllActiveTasksThatNeedCommittingOnHandleRevocationWithEo
                 .thenReturn(singletonList(task10));
     
             final ConsumerGroupMetadata groupMetadata = new ConsumerGroupMetadata("appId");
    -        expect(consumer.groupMetadata()).andReturn(groupMetadata);
    -        producer.commitTransaction(expectedCommittedOffsets, groupMetadata);
    -        expectLastCall();
    +        when(consumer.groupMetadata()).thenReturn(groupMetadata);
     
             task00.committedOffsets();
    -        EasyMock.expectLastCall();
             task01.committedOffsets();
    -        EasyMock.expectLastCall();
             task02.committedOffsets();
    -        EasyMock.expectLastCall();
             task10.committedOffsets();
    -        EasyMock.expectLastCall();
    -
    -        replay(consumer);
     
             taskManager.handleAssignment(assignmentActive, assignmentStandby);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -3331,6 +3226,8 @@ public void shouldCommitAllActiveTasksThatNeedCommittingOnHandleRevocationWithEo
             assertThat(task01.commitNeeded, is(false));
             assertThat(task02.commitPrepared, is(false));
             assertThat(task10.commitPrepared, is(false));
    +
    +        verify(producer).commitTransaction(expectedCommittedOffsets, groupMetadata);
         }
     
         @Test
    @@ -3364,16 +3261,12 @@ public void shouldCommitAllNeededTasksOnHandleRevocation() {
             final Map> assignmentStandby = mkMap(
                 mkEntry(taskId10, taskId10Partitions)
             );
    -        expectRestoreToBeCompleted(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
     
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive)))
    +        when(activeTaskCreator.createTasks(any(), eq(assignmentActive)))
                 .thenReturn(asList(task00, task01, task02));
             when(standbyTaskCreator.createTasks(assignmentStandby))
                 .thenReturn(singletonList(task10));
    -        consumer.commitSync(expectedCommittedOffsets);
    -        expectLastCall();
    -
    -        replay(consumer);
     
             taskManager.handleAssignment(assignmentActive, assignmentStandby);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -3390,6 +3283,8 @@ public void shouldCommitAllNeededTasksOnHandleRevocation() {
             assertThat(task01.commitPrepared, is(true));
             assertThat(task02.commitPrepared, is(false));
             assertThat(task10.commitPrepared, is(false));
    +
    +        verify(consumer).commitSync(expectedCommittedOffsets);
         }
     
         @Test
    @@ -3404,13 +3299,11 @@ public void shouldNotCommitOnHandleAssignmentIfNoTaskClosed() {
             final Map> assignmentActive = singletonMap(taskId00, taskId00Partitions);
             final Map> assignmentStandby = singletonMap(taskId10, taskId10Partitions);
     
    -        expectRestoreToBeCompleted(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
     
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive))).thenReturn(singleton(task00));
    +        when(activeTaskCreator.createTasks(any(), eq(assignmentActive))).thenReturn(singleton(task00));
             when(standbyTaskCreator.createTasks(assignmentStandby)).thenReturn(singletonList(task10));
     
    -        replay(consumer);
    -
             taskManager.handleAssignment(assignmentActive, assignmentStandby);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
             assertThat(task00.state(), is(Task.State.RUNNING));
    @@ -3434,13 +3327,11 @@ public void shouldNotCommitOnHandleAssignmentIfOnlyStandbyTaskClosed() {
             final Map> assignmentActive = singletonMap(taskId00, taskId00Partitions);
             final Map> assignmentStandby = singletonMap(taskId10, taskId10Partitions);
     
    -        expectRestoreToBeCompleted(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
     
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive))).thenReturn(singleton(task00));
    +        when(activeTaskCreator.createTasks(any(), eq(assignmentActive))).thenReturn(singleton(task00));
             when(standbyTaskCreator.createTasks(assignmentStandby)).thenReturn(singletonList(task10));
     
    -        replay(consumer);
    -
             taskManager.handleAssignment(assignmentActive, assignmentStandby);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
             assertThat(task00.state(), is(Task.State.RUNNING));
    @@ -3455,7 +3346,7 @@ public void shouldNotCommitOnHandleAssignmentIfOnlyStandbyTaskClosed() {
         public void shouldNotCommitCreatedTasksOnRevocationOrClosure() {
             final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
     
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(task00.state(), is(Task.State.CREATED));
    @@ -3465,7 +3356,7 @@ public void shouldNotCommitCreatedTasksOnRevocationOrClosure() {
     
             taskManager.handleAssignment(emptyMap(), emptyMap());
             assertThat(task00.state(), is(Task.State.CLOSED));
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(eq(taskId00));
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
         }
     
         @Test
    @@ -3478,17 +3369,14 @@ public void suspend() {
                 }
             };
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        replay(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
             assertThat(task00.state(), is(Task.State.RUNNING));
     
             assertThrows(RuntimeException.class, () -> taskManager.handleRevocation(taskId00Partitions));
             assertThat(task00.state(), is(Task.State.SUSPENDED));
    -
    -        verify(consumer);
         }
     
         @Test
    @@ -3549,7 +3437,7 @@ public void closeDirty() {
                 }
             };
     
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment)))
    +        when(activeTaskCreator.createTasks(any(), eq(assignment)))
                 .thenReturn(asList(task00, task01, task02, task03));
     
             taskManager.handleAssignment(assignment, emptyMap());
    @@ -3577,8 +3465,8 @@ public void closeDirty() {
                 )
             );
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
    -        Mockito.verify(changeLogReader).enforceRestoreActive();
    -        Mockito.verify(changeLogReader).completedChangelogs();
    +        verify(changeLogReader).enforceRestoreActive();
    +        verify(changeLogReader).completedChangelogs();
     
             final RuntimeException exception = assertThrows(
                 RuntimeException.class,
    @@ -3595,9 +3483,9 @@ public void closeDirty() {
             assertThat(task03.state(), is(Task.State.CLOSED));
             assertThat(taskManager.activeTaskMap(), Matchers.anEmptyMap());
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
    -        Mockito.verify(activeTaskCreator, times(4)).closeAndRemoveTaskProducerIfNeeded(any());
    +        verify(activeTaskCreator, times(4)).closeAndRemoveTaskProducerIfNeeded(any());
             // the active task creator should also get closed (so that it closes the thread producer if applicable)
    -        Mockito.verify(activeTaskCreator).closeThreadProducerIfNeeded();
    +        verify(activeTaskCreator).closeThreadProducerIfNeeded();
         }
     
         @Test
    @@ -3615,7 +3503,7 @@ public Set changelogPartitions() {
             final Map offsets = singletonMap(t1p0, new OffsetAndMetadata(0L, null));
             task00.setCommittableOffsetsAndMetadata(offsets);
     
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment))).thenReturn(singletonList(task00));
    +        when(activeTaskCreator.createTasks(any(), eq(assignment))).thenReturn(singletonList(task00));
             doThrow(new RuntimeException("whatever"))
                 .when(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
     
    @@ -3635,8 +3523,8 @@ public Set changelogPartitions() {
                 )
             );
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
    -        Mockito.verify(changeLogReader).enforceRestoreActive();
    -        Mockito.verify(changeLogReader).completedChangelogs();
    +        verify(changeLogReader).enforceRestoreActive();
    +        verify(changeLogReader).completedChangelogs();
     
             final RuntimeException exception = assertThrows(RuntimeException.class, () -> taskManager.shutdown(true));
     
    @@ -3644,9 +3532,9 @@ public Set changelogPartitions() {
             assertThat(exception.getCause().getMessage(), is("whatever"));
             assertThat(taskManager.activeTaskMap(), Matchers.anEmptyMap());
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
             // the active task creator should also get closed (so that it closes the thread producer if applicable)
    -        Mockito.verify(activeTaskCreator).closeThreadProducerIfNeeded();
    +        verify(activeTaskCreator).closeThreadProducerIfNeeded();
         }
     
         @Test
    @@ -3662,7 +3550,7 @@ public Set changelogPartitions() {
                 }
             };
     
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment))).thenReturn(singletonList(task00));
    +        when(activeTaskCreator.createTasks(any(), eq(assignment))).thenReturn(singletonList(task00));
             doThrow(new RuntimeException("whatever")).when(activeTaskCreator).closeThreadProducerIfNeeded();
     
             taskManager.handleAssignment(assignment, emptyMap());
    @@ -3681,8 +3569,8 @@ public Set changelogPartitions() {
                 )
             );
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
    -        Mockito.verify(changeLogReader).enforceRestoreActive();
    -        Mockito.verify(changeLogReader).completedChangelogs();
    +        verify(changeLogReader).enforceRestoreActive();
    +        verify(changeLogReader).completedChangelogs();
     
             final RuntimeException exception = assertThrows(RuntimeException.class, () -> taskManager.shutdown(true));
     
    @@ -3691,7 +3579,7 @@ public Set changelogPartitions() {
             assertThat(taskManager.activeTaskMap(), Matchers.anEmptyMap());
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
             // the active task creator should also get closed (so that it closes the thread producer if applicable)
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
         }
     
         @Test
    @@ -3752,7 +3640,7 @@ public void suspend() {
             assertThat(task01.state(), is(Task.State.SUSPENDED));
             assertThat(task02.state(), is(Task.State.SUSPENDED));
     
    -        Mockito.verifyNoInteractions(activeTaskCreator);
    +        verifyNoInteractions(activeTaskCreator);
         }
     
         @Test
    @@ -3784,8 +3672,8 @@ public void suspend() {
                 }
             };
     
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment))).thenReturn(asList(task00, task01, task02));
    -        doThrow(new RuntimeException("whatever")).when(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(Mockito.any());
    +        when(activeTaskCreator.createTasks(any(), eq(assignment))).thenReturn(asList(task00, task01, task02));
    +        doThrow(new RuntimeException("whatever")).when(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(any());
             doThrow(new RuntimeException("whatever all")).when(activeTaskCreator).closeThreadProducerIfNeeded();
     
             taskManager.handleAssignment(assignment, emptyMap());
    @@ -3810,8 +3698,8 @@ public void suspend() {
                 )
             );
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
    -        Mockito.verify(changeLogReader).enforceRestoreActive();
    -        Mockito.verify(changeLogReader).completedChangelogs();
    +        verify(changeLogReader).enforceRestoreActive();
    +        verify(changeLogReader).completedChangelogs();
     
             taskManager.shutdown(false);
     
    @@ -3820,9 +3708,9 @@ public void suspend() {
             assertThat(task02.state(), is(Task.State.CLOSED));
             assertThat(taskManager.activeTaskMap(), Matchers.anEmptyMap());
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
    -        Mockito.verify(activeTaskCreator, times(3)).closeAndRemoveTaskProducerIfNeeded(any());
    +        verify(activeTaskCreator, times(3)).closeAndRemoveTaskProducerIfNeeded(any());
             // the active task creator should also get closed (so that it closes the thread producer if applicable)
    -        Mockito.verify(activeTaskCreator).closeThreadProducerIfNeeded();
    +        verify(activeTaskCreator).closeThreadProducerIfNeeded();
         }
     
         @Test
    @@ -3830,8 +3718,6 @@ public void shouldCloseStandbyTasksOnShutdown() {
             final Map> assignment = singletonMap(taskId00, taskId00Partitions);
             final Task task00 = new StateMachineTask(taskId00, taskId00Partitions, false, stateManager);
     
    -        taskManager.setMainConsumer(mockitoConsumer);
    -
             // `handleAssignment`
             when(standbyTaskCreator.createTasks(assignment)).thenReturn(singletonList(task00));
     
    @@ -3848,10 +3734,10 @@ public void shouldCloseStandbyTasksOnShutdown() {
             assertThat(taskManager.activeTaskMap(), Matchers.anEmptyMap());
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
             // the active task creator should also get closed (so that it closes the thread producer if applicable)
    -        Mockito.verify(activeTaskCreator).closeThreadProducerIfNeeded();
    +        verify(activeTaskCreator).closeThreadProducerIfNeeded();
             // `tryToCompleteRestoration`
    -        Mockito.verify(mockitoConsumer).assignment();
    -        Mockito.verify(mockitoConsumer).resume(Mockito.eq(emptySet()));
    +        verify(consumer).assignment();
    +        verify(consumer).resume(eq(emptySet()));
         }
     
         @Test
    @@ -3870,12 +3756,12 @@ public void shouldShutDownStateUpdaterAndCloseFailedTasksDirty() {
     
             taskManager.shutdown(true);
     
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(failedStatefulTask.id());
    -        Mockito.verify(activeTaskCreator).closeThreadProducerIfNeeded();
    -        Mockito.verify(stateUpdater).shutdown(Duration.ofMillis(Long.MAX_VALUE));
    -        Mockito.verify(failedStatefulTask).prepareCommit();
    -        Mockito.verify(failedStatefulTask).suspend();
    -        Mockito.verify(failedStatefulTask).closeDirty();
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(failedStatefulTask.id());
    +        verify(activeTaskCreator).closeThreadProducerIfNeeded();
    +        verify(stateUpdater).shutdown(Duration.ofMillis(Long.MAX_VALUE));
    +        verify(failedStatefulTask).prepareCommit();
    +        verify(failedStatefulTask).suspend();
    +        verify(failedStatefulTask).closeDirty();
         }
     
         @Test
    @@ -3885,7 +3771,7 @@ public void shouldShutdownSchedulingTaskManager() {
     
             taskManager.shutdown(true);
     
    -        Mockito.verify(schedulingTaskManager).shutdown(Duration.ofMillis(Long.MAX_VALUE));
    +        verify(schedulingTaskManager).shutdown(Duration.ofMillis(Long.MAX_VALUE));
         }
     
         @Test
    @@ -3903,13 +3789,13 @@ public void shouldShutDownStateUpdaterAndAddRestoredTasksToTaskRegistry() {
     
             taskManager.shutdown(true);
     
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(statefulTask1.id());
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(statefulTask2.id());
    -        Mockito.verify(activeTaskCreator).closeThreadProducerIfNeeded();
    -        Mockito.verify(stateUpdater).shutdown(Duration.ofMillis(Long.MAX_VALUE));
    -        Mockito.verify(tasks).addActiveTasks(restoredTasks);
    -        Mockito.verify(statefulTask1).closeClean();
    -        Mockito.verify(statefulTask2).closeClean();
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(statefulTask1.id());
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(statefulTask2.id());
    +        verify(activeTaskCreator).closeThreadProducerIfNeeded();
    +        verify(stateUpdater).shutdown(Duration.ofMillis(Long.MAX_VALUE));
    +        verify(tasks).addActiveTasks(restoredTasks);
    +        verify(statefulTask1).closeClean();
    +        verify(statefulTask2).closeClean();
         }
     
         @Test
    @@ -3926,22 +3812,22 @@ public void shouldShutDownStateUpdaterAndAddRemovedTasksToTaskRegistry() {
     
             taskManager.shutdown(true);
     
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(removedStatefulTask.id());
    -        Mockito.verify(activeTaskCreator).closeThreadProducerIfNeeded();
    -        Mockito.verify(stateUpdater).shutdown(Duration.ofMillis(Long.MAX_VALUE));
    -        Mockito.verify(tasks).addActiveTasks(mkSet(removedStatefulTask));
    -        Mockito.verify(tasks).addStandbyTasks(mkSet(removedStandbyTask));
    -        Mockito.verify(removedStatefulTask).closeClean();
    -        Mockito.verify(removedStandbyTask).closeClean();
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(removedStatefulTask.id());
    +        verify(activeTaskCreator).closeThreadProducerIfNeeded();
    +        verify(stateUpdater).shutdown(Duration.ofMillis(Long.MAX_VALUE));
    +        verify(tasks).addActiveTasks(mkSet(removedStatefulTask));
    +        verify(tasks).addStandbyTasks(mkSet(removedStandbyTask));
    +        verify(removedStatefulTask).closeClean();
    +        verify(removedStandbyTask).closeClean();
         }
     
         @Test
         public void shouldInitializeNewActiveTasks() {
             final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment)))
    +        when(consumer.assignment()).thenReturn(assignment);
    +
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment)))
                 .thenReturn(singletonList(task00));
    -        replay(consumer);
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -3950,18 +3836,16 @@ public void shouldInitializeNewActiveTasks() {
             assertThat(taskManager.activeTaskMap(), Matchers.equalTo(singletonMap(taskId00, task00)));
             assertThat(taskManager.standbyTaskMap(), Matchers.anEmptyMap());
             // verifies that we actually resume the assignment at the end of restoration.
    -        verify(consumer);
    +        verify(consumer).resume(assignment);
         }
     
         @Test
         public void shouldInitialiseNewStandbyTasks() {
             final StateMachineTask task01 = new StateMachineTask(taskId01, taskId01Partitions, false, stateManager);
     
    -        expectRestoreToBeCompleted(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
             when(standbyTaskCreator.createTasks(taskId01Assignment)).thenReturn(singletonList(task01));
     
    -        replay(consumer);
    -
             taskManager.handleAssignment(emptyMap(), taskId01Assignment);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
     
    @@ -3972,17 +3856,14 @@ public void shouldInitialiseNewStandbyTasks() {
     
         @Test
         public void shouldHandleRebalanceEvents() {
    -        final Set assignment = singleton(new TopicPartition("assignment", 0));
    -        taskManager.setMainConsumer(mockitoConsumer);
    -        when(mockitoConsumer.assignment()).thenReturn(assignment);
    -        expect(stateDirectory.listNonEmptyTaskDirectories()).andReturn(new ArrayList<>());
    -        replay(stateDirectory);
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(stateDirectory.listNonEmptyTaskDirectories()).thenReturn(new ArrayList<>());
             assertThat(taskManager.rebalanceInProgress(), is(false));
             taskManager.handleRebalanceStart(emptySet());
             assertThat(taskManager.rebalanceInProgress(), is(true));
             taskManager.handleRebalanceComplete();
             assertThat(taskManager.rebalanceInProgress(), is(false));
    -        Mockito.verify(mockitoConsumer).pause(assignment);
    +        verify(consumer).pause(assignment);
         }
     
         @Test
    @@ -3992,15 +3873,11 @@ public void shouldCommitActiveAndStandbyTasks() {
             task00.setCommittableOffsetsAndMetadata(offsets);
             final StateMachineTask task01 = new StateMachineTask(taskId01, taskId01Partitions, false, stateManager);
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment)))
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment)))
                 .thenReturn(singletonList(task00));
             when(standbyTaskCreator.createTasks(taskId01Assignment))
                 .thenReturn(singletonList(task01));
    -        consumer.commitSync(offsets);
    -        expectLastCall();
    -
    -        replay(consumer);
     
             taskManager.handleAssignment(taskId00Assignment, taskId01Assignment);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -4014,6 +3891,8 @@ public void shouldCommitActiveAndStandbyTasks() {
             assertThat(taskManager.commitAll(), equalTo(2));
             assertThat(task00.commitNeeded, is(false));
             assertThat(task01.commitNeeded, is(false));
    +
    +        verify(consumer).commitSync(offsets);
         }
     
         @Test
    @@ -4036,16 +3915,12 @@ public void shouldCommitProvidedTasksIfNeeded() {
                 mkEntry(taskId05, taskId05Partitions)
             );
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive)))
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(assignmentActive)))
                 .thenReturn(Arrays.asList(task00, task01, task02));
             when(standbyTaskCreator.createTasks(assignmentStandby))
                 .thenReturn(Arrays.asList(task03, task04, task05));
     
    -        consumer.commitSync(eq(emptyMap()));
    -
    -        replay(consumer);
    -
             taskManager.handleAssignment(assignmentActive, assignmentStandby);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
     
    @@ -4070,11 +3945,9 @@ public void shouldCommitProvidedTasksIfNeeded() {
         public void shouldNotCommitOffsetsIfOnlyStandbyTasksAssigned() {
             final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, false, stateManager);
     
    -        expectRestoreToBeCompleted(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
             when(standbyTaskCreator.createTasks(taskId00Assignment)).thenReturn(singletonList(task00));
     
    -        replay(consumer);
    -
             taskManager.handleAssignment(Collections.emptyMap(), taskId00Assignment);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
     
    @@ -4094,14 +3967,12 @@ public void shouldNotCommitActiveAndStandbyTasksWhileRebalanceInProgress() throw
             makeTaskFolders(taskId00.toString(), taskId01.toString());
             expectDirectoryNotEmpty(taskId00, taskId01);
             expectLockObtainedFor(taskId00, taskId01);
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment)))
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment)))
                 .thenReturn(singletonList(task00));
             when(standbyTaskCreator.createTasks(taskId01Assignment))
                 .thenReturn(singletonList(task01));
     
    -        replay(stateDirectory, consumer);
    -
             taskManager.handleAssignment(taskId00Assignment, taskId01Assignment);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
     
    @@ -4130,17 +4001,16 @@ public void shouldCommitViaConsumerIfEosDisabled() {
             final Map offsets = singletonMap(t1p1, new OffsetAndMetadata(0L, null));
             task01.setCommittableOffsetsAndMetadata(offsets);
             task01.setCommitNeeded();
    -        taskManager.setMainConsumer(mockitoConsumer);
             taskManager.addTask(task01);
     
             taskManager.commitAll();
     
    -        Mockito.verify(mockitoConsumer).commitSync(offsets);
    +        verify(consumer).commitSync(offsets);
         }
     
         @Test
         public void shouldCommitViaProducerIfEosAlphaEnabled() {
    -        final StreamsProducer producer = Mockito.mock(StreamsProducer.class);
    +        final StreamsProducer producer = mock(StreamsProducer.class);
             when(activeTaskCreator.streamsProducerForTask(any(TaskId.class)))
                 .thenReturn(producer);
     
    @@ -4149,14 +4019,14 @@ public void shouldCommitViaProducerIfEosAlphaEnabled() {
     
             shouldCommitViaProducerIfEosEnabled(ProcessingMode.EXACTLY_ONCE_ALPHA, offsetsT01, offsetsT02);
     
    -        Mockito.verify(producer).commitTransaction(offsetsT01, new ConsumerGroupMetadata("appId"));
    -        Mockito.verify(producer).commitTransaction(offsetsT02, new ConsumerGroupMetadata("appId"));
    -        Mockito.verifyNoMoreInteractions(producer);
    +        verify(producer).commitTransaction(offsetsT01, new ConsumerGroupMetadata("appId"));
    +        verify(producer).commitTransaction(offsetsT02, new ConsumerGroupMetadata("appId"));
    +        verifyNoMoreInteractions(producer);
         }
     
         @Test
         public void shouldCommitViaProducerIfEosV2Enabled() {
    -        final StreamsProducer producer = Mockito.mock(StreamsProducer.class);
    +        final StreamsProducer producer = mock(StreamsProducer.class);
             when(activeTaskCreator.threadProducer()).thenReturn(producer);
     
             final Map offsetsT01 = singletonMap(t1p1, new OffsetAndMetadata(0L, null));
    @@ -4167,15 +4037,14 @@ public void shouldCommitViaProducerIfEosV2Enabled() {
     
             shouldCommitViaProducerIfEosEnabled(ProcessingMode.EXACTLY_ONCE_V2, offsetsT01, offsetsT02);
     
    -        Mockito.verify(producer).commitTransaction(allOffsets, new ConsumerGroupMetadata("appId"));
    -        Mockito.verifyNoMoreInteractions(producer);
    +        verify(producer).commitTransaction(allOffsets, new ConsumerGroupMetadata("appId"));
    +        verifyNoMoreInteractions(producer);
         }
     
         private void shouldCommitViaProducerIfEosEnabled(final ProcessingMode processingMode,
                                                          final Map offsetsT01,
                                                          final Map offsetsT02) {
             final TaskManager taskManager = setUpTaskManager(processingMode, false);
    -        taskManager.setMainConsumer(mockitoConsumer);
     
             final StateMachineTask task01 = new StateMachineTask(taskId01, taskId01Partitions, true, stateManager);
             task01.setCommittableOffsetsAndMetadata(offsetsT01);
    @@ -4186,7 +4055,7 @@ private void shouldCommitViaProducerIfEosEnabled(final ProcessingMode processing
             task02.setCommitNeeded();
             taskManager.addTask(task02);
     
    -        when(mockitoConsumer.groupMetadata()).thenReturn(new ConsumerGroupMetadata("appId"));
    +        when(consumer.groupMetadata()).thenReturn(new ConsumerGroupMetadata("appId"));
     
             taskManager.commitAll();
         }
    @@ -4200,9 +4069,8 @@ public Map prepareCommit() {
                 }
             };
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        replay(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -4225,11 +4093,9 @@ public Map prepareCommit() {
                 }
             };
     
    -        expectRestoreToBeCompleted(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
             when(standbyTaskCreator.createTasks(taskId01Assignment)).thenReturn(singletonList(task01));
     
    -        replay(consumer);
    -
             taskManager.handleAssignment(emptyMap(), taskId01Assignment);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
     
    @@ -4249,7 +4115,7 @@ public void shouldSendPurgeData() {
             when(adminClient.deleteRecords(singletonMap(t1p1, RecordsToDelete.beforeOffset(17L))))
                 .thenReturn(new DeleteRecordsResult(singletonMap(t1p1, completedFuture())));
     
    -        final InOrder inOrder = Mockito.inOrder(adminClient);
    +        final InOrder inOrder = inOrder(adminClient);
     
             final Map purgableOffsets = new HashMap<>();
             final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager) {
    @@ -4259,10 +4125,8 @@ public Map purgeableOffsets() {
                 }
             };
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -
    -        replay(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -4294,9 +4158,8 @@ public Map purgeableOffsets() {
                 }
             };
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        replay(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -4317,15 +4180,13 @@ public Map purgeableOffsets() {
         public void shouldIgnorePurgeDataErrors() {
             final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
     
    -        expectRestoreToBeCompleted(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
     
             final KafkaFutureImpl futureDeletedRecords = new KafkaFutureImpl<>();
             final DeleteRecordsResult deleteRecordsResult = new DeleteRecordsResult(singletonMap(t1p1, futureDeletedRecords));
             futureDeletedRecords.completeExceptionally(new Exception("KABOOM!"));
             when(adminClient.deleteRecords(any())).thenReturn(deleteRecordsResult);
     
    -        replay(consumer);
    -
             taskManager.addTask(task00);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
     
    @@ -4366,15 +4227,11 @@ public void shouldMaybeCommitAllActiveTasksThatNeedCommit() {
                 mkEntry(taskId10, taskId10Partitions)
             );
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignmentActive)))
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(assignmentActive)))
                 .thenReturn(asList(task00, task01, task02, task03));
             when(standbyTaskCreator.createTasks(assignmentStandby))
                 .thenReturn(singletonList(task04));
    -        consumer.commitSync(expectedCommittedOffsets);
    -        expectLastCall();
    -
    -        replay(consumer);
     
             taskManager.handleAssignment(assignmentActive, assignmentStandby);
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -4399,6 +4256,8 @@ public void shouldMaybeCommitAllActiveTasksThatNeedCommit() {
             task04.setCommitRequested();
     
             assertThat(taskManager.maybeCommitActiveTasksPerUserRequested(), equalTo(3));
    +
    +        verify(consumer).commitSync(expectedCommittedOffsets);
         }
     
         @Test
    @@ -4406,16 +4265,15 @@ public void shouldProcessActiveTasks() {
             final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager);
             final StateMachineTask task01 = new StateMachineTask(taskId01, taskId01Partitions, true, stateManager);
     
    -        final Map> assignment = new HashMap<>();
    -        assignment.put(taskId00, taskId00Partitions);
    -        assignment.put(taskId01, taskId01Partitions);
    +        final Map> firstAssignment = new HashMap<>();
    +        firstAssignment.put(taskId00, taskId00Partitions);
    +        firstAssignment.put(taskId01, taskId01Partitions);
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment)))
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(firstAssignment)))
                 .thenReturn(Arrays.asList(task00, task01));
    -        replay(consumer);
     
    -        taskManager.handleAssignment(assignment, emptyMap());
    +        taskManager.handleAssignment(firstAssignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
     
             assertThat(task00.state(), is(Task.State.RUNNING));
    @@ -4523,9 +4381,8 @@ public boolean process(final long wallClockTime) {
                 }
             };
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        replay(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -4547,10 +4404,9 @@ public boolean process(final long wallClockTime) {
                 }
             };
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment)))
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment)))
                 .thenReturn(singletonList(task00));
    -        replay(consumer);
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -4575,9 +4431,8 @@ public boolean maybePunctuateStreamTime() {
                 }
             };
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        replay(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -4596,9 +4451,8 @@ public boolean maybePunctuateStreamTime() {
                 }
             };
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        replay(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -4622,9 +4476,8 @@ public boolean maybePunctuateSystemTime() {
                 }
             };
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        replay(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(true));
    @@ -4644,14 +4497,12 @@ public Set changelogPartitions() {
                 }
             };
     
    -        taskManager.setMainConsumer(mockitoConsumer);
    -
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             taskManager.handleAssignment(taskId00Assignment, emptyMap());
             assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), null), is(false));
             assertThat(task00.state(), is(Task.State.RESTORING));
    -        Mockito.verifyNoInteractions(mockitoConsumer);
    +        verifyNoInteractions(consumer);
         }
     
         @Test
    @@ -4660,12 +4511,8 @@ public void shouldHaveRemainingPartitionsUncleared() {
             final Map offsets = singletonMap(t1p0, new OffsetAndMetadata(0L, null));
             task00.setCommittableOffsetsAndMetadata(offsets);
     
    -        expectRestoreToBeCompleted(consumer);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(task00));
    -        consumer.commitSync(offsets);
    -        expectLastCall();
    -
    -        replay(consumer);
    +        when(consumer.assignment()).thenReturn(assignment);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(task00));
     
             try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(TaskManager.class)) {
                 appender.setClassLoggerToDebug(TaskManager.class);
    @@ -4819,10 +4666,9 @@ private Map handleAssignment(final Map handleAssignment(final Map consumer) {
    -        final Set assignment = singleton(new TopicPartition("assignment", 0));
    -        expect(consumer.assignment()).andReturn(assignment);
    -        consumer.pause(assignment);
    -    }
    -
         @Test
         public void shouldThrowTaskMigratedExceptionOnCommitFailed() {
             final StateMachineTask task01 = new StateMachineTask(taskId01, taskId01Partitions, true, stateManager);
             final Map offsets = singletonMap(t1p0, new OffsetAndMetadata(0L, null));
             task01.setCommittableOffsetsAndMetadata(offsets);
             task01.setCommitNeeded();
    -        taskManager.setMainConsumer(mockitoConsumer);
             taskManager.addTask(task01);
     
    -        doThrow(new CommitFailedException()).when(mockitoConsumer).commitSync(offsets);
    +        doThrow(new CommitFailedException()).when(consumer).commitSync(offsets);
     
             final TaskMigratedException thrown = assertThrows(
                 TaskMigratedException.class,
    @@ -4910,9 +4742,7 @@ public void shouldNotFailForTimeoutExceptionOnConsumerCommit() {
             task00.setCommittableOffsetsAndMetadata(taskId00Partitions.stream().collect(Collectors.toMap(p -> p, p -> new OffsetAndMetadata(0))));
             task01.setCommittableOffsetsAndMetadata(taskId00Partitions.stream().collect(Collectors.toMap(p -> p, p -> new OffsetAndMetadata(0))));
     
    -        taskManager.setMainConsumer(mockitoConsumer);
    -
    -        doThrow(new TimeoutException("KABOOM!")).doNothing().when(mockitoConsumer).commitSync(any(Map.class));
    +        doThrow(new TimeoutException("KABOOM!")).doNothing().when(consumer).commitSync(any(Map.class));
     
             task00.setCommitNeeded();
     
    @@ -4924,14 +4754,13 @@ public void shouldNotFailForTimeoutExceptionOnConsumerCommit() {
             assertNull(task00.timeout);
             assertNull(task01.timeout);
     
    -        Mockito.verify(mockitoConsumer, times(2)).commitSync(any(Map.class));
    +        verify(consumer, times(2)).commitSync(any(Map.class));
         }
     
         @Test
         public void shouldNotFailForTimeoutExceptionOnCommitWithEosAlpha() {
             final Tasks tasks = mock(Tasks.class);
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.EXACTLY_ONCE_ALPHA, tasks, false);
    -        taskManager.setMainConsumer(mockitoConsumer);
     
             final StreamsProducer producer = mock(StreamsProducer.class);
             when(activeTaskCreator.streamsProducerForTask(any(TaskId.class))).thenReturn(producer);
    @@ -4967,13 +4796,12 @@ public void shouldNotFailForTimeoutExceptionOnCommitWithEosAlpha() {
                 equalTo(Collections.singleton(taskId00))
             );
     
    -        Mockito.verify(mockitoConsumer, times(2)).groupMetadata();
    +        verify(consumer, times(2)).groupMetadata();
         }
     
         @Test
         public void shouldThrowTaskCorruptedExceptionForTimeoutExceptionOnCommitWithEosV2() {
             final TaskManager taskManager = setUpTaskManager(ProcessingMode.EXACTLY_ONCE_V2, false);
    -        taskManager.setMainConsumer(mockitoConsumer);
     
             final StreamsProducer producer = mock(StreamsProducer.class);
             when(activeTaskCreator.threadProducer()).thenReturn(producer);
    @@ -5003,7 +4831,7 @@ public void shouldThrowTaskCorruptedExceptionForTimeoutExceptionOnCommitWithEosV
                 equalTo(mkSet(taskId00, taskId01))
             );
     
    -        Mockito.verify(mockitoConsumer).groupMetadata();
    +        verify(consumer).groupMetadata();
         }
     
         @Test
    @@ -5012,10 +4840,9 @@ public void shouldStreamsExceptionOnCommitError() {
             final Map offsets = singletonMap(t1p0, new OffsetAndMetadata(0L, null));
             task01.setCommittableOffsetsAndMetadata(offsets);
             task01.setCommitNeeded();
    -        taskManager.setMainConsumer(mockitoConsumer);
             taskManager.addTask(task01);
     
    -        doThrow(new KafkaException()).when(mockitoConsumer).commitSync(offsets);
    +        doThrow(new KafkaException()).when(consumer).commitSync(offsets);
     
             final StreamsException thrown = assertThrows(
                 StreamsException.class,
    @@ -5033,10 +4860,9 @@ public void shouldFailOnCommitFatal() {
             final Map offsets = singletonMap(t1p0, new OffsetAndMetadata(0L, null));
             task01.setCommittableOffsetsAndMetadata(offsets);
             task01.setCommitNeeded();
    -        taskManager.setMainConsumer(mockitoConsumer);
             taskManager.addTask(task01);
     
    -        doThrow(new RuntimeException("KABOOM")).when(mockitoConsumer).commitSync(offsets);
    +        doThrow(new RuntimeException("KABOOM")).when(consumer).commitSync(offsets);
     
             final RuntimeException thrown = assertThrows(
                 RuntimeException.class,
    @@ -5060,9 +4886,7 @@ public void suspend() {
     
             final Map> assignment = new HashMap<>(taskId00Assignment);
             assignment.putAll(taskId01Assignment);
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(assignment))).thenReturn(asList(task00, task01));
    -
    -        taskManager.setMainConsumer(mockitoConsumer);
    +        when(activeTaskCreator.createTasks(any(), eq(assignment))).thenReturn(asList(task00, task01));
     
             taskManager.handleAssignment(assignment, Collections.emptyMap());
     
    @@ -5073,31 +4897,29 @@ public void suspend() {
             assertThat(thrown.getCause().getMessage(), is("KABOOM!"));
             assertThat(task00.state(), is(Task.State.SUSPENDED));
             assertThat(task01.state(), is(Task.State.SUSPENDED));
    -        Mockito.verifyNoInteractions(mockitoConsumer);
    +        verifyNoInteractions(consumer);
         }
     
         @Test
         public void shouldConvertActiveTaskToStandbyTask() {
    -        final StreamTask activeTask = Mockito.mock(StreamTask.class);
    +        final StreamTask activeTask = mock(StreamTask.class);
             when(activeTask.id()).thenReturn(taskId00);
             when(activeTask.inputPartitions()).thenReturn(taskId00Partitions);
             when(activeTask.isActive()).thenReturn(true);
     
    -        final StandbyTask standbyTask = Mockito.mock(StandbyTask.class);
    +        final StandbyTask standbyTask = mock(StandbyTask.class);
             when(standbyTask.id()).thenReturn(taskId00);
     
    -        when(activeTaskCreator.createTasks(any(), Mockito.eq(taskId00Assignment))).thenReturn(singletonList(activeTask));
    -        when(standbyTaskCreator.createStandbyTaskFromActive(Mockito.any(), Mockito.eq(taskId00Partitions))).thenReturn(standbyTask);
    -
    -        taskManager.setMainConsumer(mockitoConsumer);
    +        when(activeTaskCreator.createTasks(any(), eq(taskId00Assignment))).thenReturn(singletonList(activeTask));
    +        when(standbyTaskCreator.createStandbyTaskFromActive(any(), eq(taskId00Partitions))).thenReturn(standbyTask);
     
             taskManager.handleAssignment(taskId00Assignment, Collections.emptyMap());
             taskManager.handleAssignment(Collections.emptyMap(), taskId00Assignment);
     
    -        Mockito.verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
    -        Mockito.verify(activeTaskCreator).createTasks(any(), Mockito.eq(emptyMap()));
    -        Mockito.verify(standbyTaskCreator, times(2)).createTasks(Collections.emptyMap());
    -        Mockito.verifyNoInteractions(mockitoConsumer);
    +        verify(activeTaskCreator).closeAndRemoveTaskProducerIfNeeded(taskId00);
    +        verify(activeTaskCreator).createTasks(any(), eq(emptyMap()));
    +        verify(standbyTaskCreator, times(2)).createTasks(Collections.emptyMap());
    +        verifyNoInteractions(consumer);
         }
     
         @Test
    @@ -5111,17 +4933,15 @@ public void shouldConvertStandbyTaskToActiveTask() {
             when(activeTask.id()).thenReturn(taskId00);
             when(activeTask.inputPartitions()).thenReturn(taskId00Partitions);
             when(standbyTaskCreator.createTasks(taskId00Assignment)).thenReturn(singletonList(standbyTask));
    -        when(activeTaskCreator.createActiveTaskFromStandby(Mockito.eq(standbyTask), Mockito.eq(taskId00Partitions), any()))
    +        when(activeTaskCreator.createActiveTaskFromStandby(eq(standbyTask), eq(taskId00Partitions), any()))
                 .thenReturn(activeTask);
     
    -        taskManager.setMainConsumer(mockitoConsumer);
    -
             taskManager.handleAssignment(Collections.emptyMap(), taskId00Assignment);
             taskManager.handleAssignment(taskId00Assignment, Collections.emptyMap());
     
    -        Mockito.verify(activeTaskCreator, times(2)).createTasks(any(), Mockito.eq(emptyMap()));
    -        Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    -        Mockito.verifyNoInteractions(mockitoConsumer);
    +        verify(activeTaskCreator, times(2)).createTasks(any(), eq(emptyMap()));
    +        verify(standbyTaskCreator).createTasks(Collections.emptyMap());
    +        verifyNoInteractions(consumer);
         }
     
         @Test
    @@ -5135,13 +4955,6 @@ public void shouldListNotPausedTasks() {
             assertEquals(taskManager.notPausedTasks().size(), 0);
         }
     
    -    private static void expectRestoreToBeCompleted(final Consumer consumer) {
    -        final Set assignment = singleton(new TopicPartition("assignment", 0));
    -        expect(consumer.assignment()).andReturn(assignment);
    -        consumer.resume(assignment);
    -        expectLastCall();
    -    }
    -
         private static KafkaFutureImpl completedFuture() {
             final KafkaFutureImpl futureDeletedRecords = new KafkaFutureImpl<>();
             futureDeletedRecords.complete(null);
    @@ -5153,14 +4966,14 @@ private void makeTaskFolders(final String... names) throws Exception {
             for (int i = 0; i < names.length; ++i) {
                 taskFolders.add(new TaskDirectory(testFolder.newFolder(names[i]), null));
             }
    -        expect(stateDirectory.listNonEmptyTaskDirectories()).andReturn(taskFolders).once();
    +        when(stateDirectory.listNonEmptyTaskDirectories()).thenReturn(taskFolders);
         }
     
         private void writeCheckpointFile(final TaskId task, final Map offsets) throws Exception {
             final File checkpointFile = getCheckpointFile(task);
             Files.createFile(checkpointFile.toPath());
             new OffsetCheckpoint(checkpointFile).write(offsets);
    -        expect(stateDirectory.checkpointFileFor(task)).andReturn(checkpointFile);
    +        lenient().when(stateDirectory.checkpointFileFor(task)).thenReturn(checkpointFile);
             expectDirectoryNotEmpty(task);
         }
     
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorFactoryTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorFactoryTest.java
    index 4c4b35f83a..ef3f0fbf4c 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorFactoryTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StandbyTaskAssignorFactoryTest.java
    @@ -31,7 +31,7 @@
     import org.mockito.quality.Strictness;
     
     import static java.util.Collections.singletonList;
    -import static org.junit.Assert.assertTrue;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     import static org.mockito.Mockito.mock;
     import static org.mockito.Mockito.never;
     import static org.mockito.Mockito.times;
    @@ -80,7 +80,7 @@ public void setUp() {
         @Test
         public void shouldReturnClientTagAwareStandbyTaskAssignorWhenRackAwareAssignmentTagsIsSet() {
             final StandbyTaskAssignor standbyTaskAssignor = StandbyTaskAssignorFactory.create(newAssignmentConfigs(singletonList("az")), rackAwareTaskAssignor);
    -        assertTrue(standbyTaskAssignor instanceof ClientTagAwareStandbyTaskAssignor);
    +        assertInstanceOf(ClientTagAwareStandbyTaskAssignor.class, standbyTaskAssignor);
             if (state != State.NULL) {
                 verify(rackAwareTaskAssignor, never()).racksForProcess();
                 verify(rackAwareTaskAssignor, never()).validClientRack();
    @@ -91,15 +91,15 @@ public void shouldReturnClientTagAwareStandbyTaskAssignorWhenRackAwareAssignment
         public void shouldReturnDefaultOrRackAwareStandbyTaskAssignorWhenRackAwareAssignmentTagsIsEmpty() {
             final StandbyTaskAssignor standbyTaskAssignor = StandbyTaskAssignorFactory.create(newAssignmentConfigs(Collections.emptyList()), rackAwareTaskAssignor);
             if (state == State.ENABLED) {
    -            assertTrue(standbyTaskAssignor instanceof ClientTagAwareStandbyTaskAssignor);
    +            assertInstanceOf(ClientTagAwareStandbyTaskAssignor.class, standbyTaskAssignor);
                 verify(rackAwareTaskAssignor, times(1)).racksForProcess();
                 verify(rackAwareTaskAssignor, times(1)).validClientRack();
             } else if (state == State.DISABLED) {
    -            assertTrue(standbyTaskAssignor instanceof DefaultStandbyTaskAssignor);
    +            assertInstanceOf(DefaultStandbyTaskAssignor.class, standbyTaskAssignor);
                 verify(rackAwareTaskAssignor, never()).racksForProcess();
                 verify(rackAwareTaskAssignor, times(1)).validClientRack();
             } else {
    -            assertTrue(standbyTaskAssignor instanceof DefaultStandbyTaskAssignor);
    +            assertInstanceOf(DefaultStandbyTaskAssignor.class, standbyTaskAssignor);
             }
         }
     
    diff --git a/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java b/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java
    index d34b3c8029..3031649944 100644
    --- a/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java
    +++ b/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java
    @@ -35,6 +35,8 @@ public class GlobalStateManagerStub implements GlobalStateManager {
         private final File baseDirectory;
         public boolean initialized;
         public boolean closed;
    +    public boolean flushed;
    +    public boolean checkpointWritten;
     
         public GlobalStateManagerStub(final Set storeNames,
                                       final Map offsets,
    @@ -64,7 +66,9 @@ public void registerStore(final StateStore store,
                                   final CommitCallback checkpoint) {}
     
         @Override
    -    public void flush() {}
    +    public void flush() {
    +        flushed = true;
    +    }
     
         @Override
         public void close() {
    @@ -77,7 +81,9 @@ public void updateChangelogOffsets(final Map writtenOffset
         }
     
         @Override
    -    public void checkpoint() {}
    +    public void checkpoint() {
    +        checkpointWritten = true;
    +    }
     
         @Override
         public StateStore getStore(final String name) {
    diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java
    index 9766d1c0fe..8c1812ccf1 100644
    --- a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java
    +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java
    @@ -32,7 +32,6 @@
     public class MockProcessor implements Processor {
         private final MockApiProcessor delegate;
     
    -
         public MockProcessor(final PunctuationType punctuationType,
                              final long scheduleInterval) {
             delegate = new MockApiProcessor<>(punctuationType, scheduleInterval);
    @@ -43,12 +42,12 @@ public MockProcessor() {
         }
     
         @Override
    -    public void init(ProcessorContext context) {
    +    public void init(final ProcessorContext context) {
             delegate.init(context);
         }
     
         @Override
    -    public void process(Record record) {
    +    public void process(final Record record) {
             delegate.process(record);
         }
     
    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 20abaa5407..5767ed9d20 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
    @@ -459,7 +459,9 @@ private void setupGlobalTask(final Time mockWallClockTime,
                     globalTopology,
                     globalProcessorContext,
                     globalStateManager,
    -                new LogAndContinueExceptionHandler()
    +                new LogAndContinueExceptionHandler(),
    +                mockWallClockTime,
    +                streamsConfig.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG)
                 );
                 globalStateTask.initialize();
                 globalProcessorContext.setRecordContext(null);
    diff --git a/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
    new file mode 100644
    index 0000000000..dc0ad4d560
    --- /dev/null
    +++ b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
    @@ -0,0 +1,299 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements. See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License. You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF 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.tests;
    +
    +import org.apache.kafka.common.serialization.Serdes;
    +import org.apache.kafka.common.utils.Bytes;
    +import org.apache.kafka.common.utils.KafkaThread;
    +import org.apache.kafka.common.utils.Utils;
    +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.Topology;
    +import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler;
    +import org.apache.kafka.streams.kstream.Consumed;
    +import org.apache.kafka.streams.kstream.Grouped;
    +import org.apache.kafka.streams.kstream.KGroupedStream;
    +import org.apache.kafka.streams.kstream.KStream;
    +import org.apache.kafka.streams.kstream.KTable;
    +import org.apache.kafka.streams.kstream.Materialized;
    +import org.apache.kafka.streams.kstream.Produced;
    +import org.apache.kafka.streams.kstream.Suppressed.BufferConfig;
    +import org.apache.kafka.streams.kstream.TimeWindows;
    +import org.apache.kafka.streams.kstream.Windowed;
    +import org.apache.kafka.streams.state.Stores;
    +import org.apache.kafka.streams.state.WindowStore;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.nio.file.Files;
    +import java.time.Duration;
    +import java.time.Instant;
    +import java.util.Properties;
    +import java.util.concurrent.CountDownLatch;
    +import java.util.concurrent.TimeUnit;
    +
    +import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses;
    +
    +public class SmokeTestClient extends SmokeTestUtil {
    +
    +    private final String name;
    +
    +    private KafkaStreams streams;
    +    private boolean uncaughtException = false;
    +    private boolean started;
    +    private volatile boolean closed;
    +
    +    private static void addShutdownHook(final String name, final Runnable runnable) {
    +        if (name != null) {
    +            Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable));
    +        } else {
    +            Runtime.getRuntime().addShutdownHook(new Thread(runnable));
    +        }
    +    }
    +
    +    private static File tempDirectory() {
    +        final String prefix = "kafka-";
    +        final File file;
    +        try {
    +            file = Files.createTempDirectory(prefix).toFile();
    +        } catch (final IOException ex) {
    +            throw new RuntimeException("Failed to create a temp dir", ex);
    +        }
    +        file.deleteOnExit();
    +
    +        addShutdownHook("delete-temp-file-shutdown-hook", () -> {
    +            try {
    +                Utils.delete(file);
    +            } catch (final IOException e) {
    +                System.out.println("Error deleting " + file.getAbsolutePath());
    +                e.printStackTrace(System.out);
    +            }
    +        });
    +
    +        return file;
    +    }
    +
    +    public SmokeTestClient(final String name) {
    +        this.name = name;
    +    }
    +
    +    public boolean started() {
    +        return started;
    +    }
    +
    +    public boolean closed() {
    +        return closed;
    +    }
    +
    +    public void start(final Properties streamsProperties) {
    +        final Topology build = getTopology();
    +        streams = new KafkaStreams(build, getStreamsConfig(streamsProperties));
    +
    +        final CountDownLatch countDownLatch = new CountDownLatch(1);
    +        streams.setStateListener((newState, oldState) -> {
    +            System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState);
    +            if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) {
    +                started = true;
    +                countDownLatch.countDown();
    +            }
    +
    +            if (newState == KafkaStreams.State.NOT_RUNNING) {
    +                closed = true;
    +            }
    +        });
    +
    +        streams.setUncaughtExceptionHandler(e -> {
    +            System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION");
    +            System.out.println(name + ": FATAL: An unexpected exception is encountered: " + e);
    +            e.printStackTrace(System.out);
    +            uncaughtException = true;
    +            return StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
    +        });
    +
    +        addShutdownHook("streams-shutdown-hook", this::close);
    +
    +        streams.start();
    +        try {
    +            if (!countDownLatch.await(1, TimeUnit.MINUTES)) {
    +                System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute");
    +            }
    +        } catch (final InterruptedException e) {
    +            System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e);
    +            e.printStackTrace(System.out);
    +        }
    +        System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED");
    +        System.out.println(name + " started at " + Instant.now());
    +    }
    +
    +    public void closeAsync() {
    +        streams.close(Duration.ZERO);
    +    }
    +
    +    public void close() {
    +        final boolean closed = streams.close(Duration.ofMinutes(1));
    +
    +        if (closed && !uncaughtException) {
    +            System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED");
    +        } else if (closed) {
    +            System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION");
    +        } else {
    +            System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close");
    +        }
    +    }
    +
    +    private Properties getStreamsConfig(final Properties props) {
    +        final Properties fullProps = new Properties(props);
    +        fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest");
    +        fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name);
    +        fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath());
    +        fullProps.putAll(props);
    +        return fullProps;
    +    }
    +
    +    public Topology getTopology() {
    +        final StreamsBuilder builder = new StreamsBuilder();
    +        final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde);
    +        final KStream source = builder.stream("data", stringIntConsumed);
    +        source.filterNot((k, v) -> k.equals("flush"))
    +              .to("echo", Produced.with(stringSerde, intSerde));
    +        final KStream data = source.filter((key, value) -> value == null || value != END);
    +        data.process(SmokeTestUtil.printProcessorSupplier("data", name));
    +
    +        // min
    +        final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde));
    +
    +        final KTable, Integer> minAggregation = groupedData
    +            .windowedBy(TimeWindows.ofSizeAndGrace(Duration.ofDays(1), Duration.ofMinutes(1)))
    +            .aggregate(
    +                () -> Integer.MAX_VALUE,
    +                (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate,
    +                Materialized
    +                    .>as("uwin-min")
    +                    .withValueSerde(intSerde)
    +                    .withRetention(Duration.ofHours(25))
    +            );
    +
    +        streamify(minAggregation, "min-raw");
    +
    +        streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed");
    +
    +        minAggregation
    +            .toStream(new Unwindow<>())
    +            .filterNot((k, v) -> k.equals("flush"))
    +            .to("min", Produced.with(stringSerde, intSerde));
    +
    +        final KTable, Integer> smallWindowSum = groupedData
    +            .windowedBy(TimeWindows.ofSizeAndGrace(Duration.ofSeconds(2), Duration.ofSeconds(30)).advanceBy(Duration.ofSeconds(1)))
    +            .reduce(Integer::sum);
    +
    +        streamify(smallWindowSum, "sws-raw");
    +        streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed");
    +
    +        final KTable minTable = builder.table(
    +            "min",
    +            Consumed.with(stringSerde, intSerde),
    +            Materialized.as("minStoreName"));
    +
    +        minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name));
    +
    +        // max
    +        groupedData
    +            .windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofDays(2)))
    +            .aggregate(
    +                () -> Integer.MIN_VALUE,
    +                (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate,
    +                Materialized.>as("uwin-max").withValueSerde(intSerde))
    +            .toStream(new Unwindow<>())
    +            .filterNot((k, v) -> k.equals("flush"))
    +            .to("max", Produced.with(stringSerde, intSerde));
    +
    +        final KTable maxTable = builder.table(
    +            "max",
    +            Consumed.with(stringSerde, intSerde),
    +            Materialized.as("maxStoreName"));
    +        maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name));
    +
    +        // sum
    +        groupedData
    +            .windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofDays(2)))
    +            .aggregate(
    +                () -> 0L,
    +                (aggKey, value, aggregate) -> (long) value + aggregate,
    +                Materialized.>as("win-sum").withValueSerde(longSerde))
    +            .toStream(new Unwindow<>())
    +            .filterNot((k, v) -> k.equals("flush"))
    +            .to("sum", Produced.with(stringSerde, longSerde));
    +
    +        final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde);
    +        final KTable sumTable = builder.table("sum", stringLongConsumed);
    +        sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name));
    +
    +        // cnt
    +        groupedData
    +            .windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofDays(2)))
    +            .count(Materialized.as("uwin-cnt"))
    +            .toStream(new Unwindow<>())
    +            .filterNot((k, v) -> k.equals("flush"))
    +            .to("cnt", Produced.with(stringSerde, longSerde));
    +
    +        final KTable cntTable = builder.table(
    +            "cnt",
    +            Consumed.with(stringSerde, longSerde),
    +            Materialized.as("cntStoreName"));
    +        cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name));
    +
    +        // dif
    +        maxTable
    +            .join(
    +                minTable,
    +                (value1, value2) -> value1 - value2)
    +            .toStream()
    +            .filterNot((k, v) -> k.equals("flush"))
    +            .to("dif", Produced.with(stringSerde, intSerde));
    +
    +        // avg
    +        sumTable
    +            .join(
    +                cntTable,
    +                (value1, value2) -> (double) value1 / (double) value2)
    +            .toStream()
    +            .filterNot((k, v) -> k.equals("flush"))
    +            .to("avg", Produced.with(stringSerde, doubleSerde));
    +
    +        // test repartition
    +        final Agg agg = new Agg();
    +        cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde))
    +                .aggregate(agg.init(), agg.adder(), agg.remover(),
    +                           Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt"))
    +                               .withKeySerde(Serdes.String())
    +                               .withValueSerde(Serdes.Long()))
    +                .toStream()
    +                .to("tagg", Produced.with(stringSerde, longSerde));
    +
    +        return builder.build();
    +    }
    +
    +    private static void streamify(final KTable, Integer> windowedTable, final String topic) {
    +        windowedTable
    +            .toStream()
    +            .filterNot((k, v) -> k.key().equals("flush"))
    +            .map((key, value) -> new KeyValue<>(key.toString(), value))
    +            .to(topic, Produced.with(stringSerde, intSerde));
    +    }
    +}
    diff --git a/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java
    new file mode 100644
    index 0000000000..dbacbb9625
    --- /dev/null
    +++ b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java
    @@ -0,0 +1,670 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements. See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License. You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF 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.tests;
    +
    +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.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.TimeoutException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.apache.kafka.common.serialization.Deserializer;
    +import org.apache.kafka.common.serialization.Serde;
    +import org.apache.kafka.common.serialization.StringDeserializer;
    +import org.apache.kafka.common.utils.Exit;
    +import org.apache.kafka.common.utils.Utils;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.PrintStream;
    +import java.nio.charset.StandardCharsets;
    +import java.time.Duration;
    +import java.time.Instant;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Random;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.function.Function;
    +import java.util.stream.Collectors;
    +import java.util.stream.Stream;
    +
    +import static java.util.Collections.emptyMap;
    +import static org.apache.kafka.common.utils.Utils.mkEntry;
    +
    +public class SmokeTestDriver extends SmokeTestUtil {
    +    private static final String[] NUMERIC_VALUE_TOPICS = {
    +        "data",
    +        "echo",
    +        "max",
    +        "min", "min-suppressed", "min-raw",
    +        "dif",
    +        "sum",
    +        "sws-raw", "sws-suppressed",
    +        "cnt",
    +        "avg",
    +        "tagg"
    +    };
    +    private static final String[] STRING_VALUE_TOPICS = {
    +        "fk"
    +    };
    +
    +    private static final String[] TOPICS = new String[NUMERIC_VALUE_TOPICS.length + STRING_VALUE_TOPICS.length];
    +    static {
    +        System.arraycopy(NUMERIC_VALUE_TOPICS, 0, TOPICS, 0, NUMERIC_VALUE_TOPICS.length);
    +        System.arraycopy(STRING_VALUE_TOPICS, 0, TOPICS, NUMERIC_VALUE_TOPICS.length, STRING_VALUE_TOPICS.length);
    +    }
    +
    +    private static final int MAX_RECORD_EMPTY_RETRIES = 30;
    +
    +    private static class ValueList {
    +        public final String key;
    +        private final int[] values;
    +        private int index;
    +
    +        ValueList(final int min, final int max) {
    +            key = min + "-" + max;
    +
    +            values = new int[max - min + 1];
    +            for (int i = 0; i < values.length; i++) {
    +                values[i] = min + i;
    +            }
    +            // We want to randomize the order of data to test not completely predictable processing order
    +            // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp)
    +            // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window
    +            shuffle(values, 10);
    +
    +            index = 0;
    +        }
    +
    +        int next() {
    +            return (index < values.length) ? values[index++] : -1;
    +        }
    +    }
    +
    +    public static String[] topics() {
    +        return Arrays.copyOf(TOPICS, TOPICS.length);
    +    }
    +
    +    static void generatePerpetually(final String kafka,
    +                                    final int numKeys,
    +                                    final int maxRecordsPerKey) {
    +        final Properties producerProps = generatorProperties(kafka);
    +
    +        int numRecordsProduced = 0;
    +
    +        final ValueList[] data = new ValueList[numKeys];
    +        for (int i = 0; i < numKeys; i++) {
    +            data[i] = new ValueList(i, i + maxRecordsPerKey - 1);
    +        }
    +
    +        final Random rand = new Random();
    +
    +        try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) {
    +            while (true) {
    +                final int index = rand.nextInt(numKeys);
    +                final String key = data[index].key;
    +                final int value = data[index].next();
    +
    +                final ProducerRecord record =
    +                    new ProducerRecord<>(
    +                        "data",
    +                        stringSerde.serializer().serialize("", key),
    +                        intSerde.serializer().serialize("", value)
    +                    );
    +                producer.send(record);
    +
    +                final ProducerRecord fkRecord =
    +                    new ProducerRecord<>(
    +                        "fk",
    +                        intSerde.serializer().serialize("", value),
    +                        stringSerde.serializer().serialize("", key)
    +                    );
    +                producer.send(fkRecord);
    +
    +                numRecordsProduced++;
    +                if (numRecordsProduced % 100 == 0) {
    +                    System.out.println(Instant.now() + " " + numRecordsProduced + " records produced");
    +                }
    +                Utils.sleep(2);
    +            }
    +        }
    +    }
    +
    +    public static Map> generate(final String kafka,
    +                                                     final int numKeys,
    +                                                     final int maxRecordsPerKey,
    +                                                     final Duration timeToSpend) {
    +        final Properties producerProps = generatorProperties(kafka);
    +
    +        int numRecordsProduced = 0;
    +
    +        final Map> allData = new HashMap<>();
    +        final ValueList[] data = new ValueList[numKeys];
    +        for (int i = 0; i < numKeys; i++) {
    +            data[i] = new ValueList(i, i + maxRecordsPerKey - 1);
    +            allData.put(data[i].key, new HashSet<>());
    +        }
    +        final Random rand = new Random();
    +
    +        int remaining = data.length;
    +
    +        final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey;
    +
    +        final List> dataNeedRetry = new ArrayList<>();
    +        final List> fkNeedRetry = new ArrayList<>();
    +
    +        try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) {
    +            while (remaining > 0) {
    +                final int index = rand.nextInt(remaining);
    +                final String key = data[index].key;
    +                final int value = data[index].next();
    +
    +                if (value < 0) {
    +                    remaining--;
    +                    data[index] = data[remaining];
    +                } else {
    +                    final ProducerRecord record =
    +                        new ProducerRecord<>(
    +                            "data",
    +                            stringSerde.serializer().serialize("", key),
    +                            intSerde.serializer().serialize("", value)
    +                        );
    +
    +                    producer.send(record, new TestCallback(record, dataNeedRetry));
    +
    +                    final ProducerRecord fkRecord =
    +                        new ProducerRecord<>(
    +                            "fk",
    +                            intSerde.serializer().serialize("", value),
    +                            stringSerde.serializer().serialize("", key)
    +                        );
    +
    +                    producer.send(fkRecord, new TestCallback(fkRecord, fkNeedRetry));
    +
    +                    numRecordsProduced++;
    +                    allData.get(key).add(value);
    +                    if (numRecordsProduced % 100 == 0) {
    +                        System.out.println(Instant.now() + " " + numRecordsProduced + " records produced");
    +                    }
    +                    Utils.sleep(Math.max(recordPauseTime, 2));
    +                }
    +            }
    +            producer.flush();
    +
    +            retry(producer, dataNeedRetry, stringSerde);
    +            retry(producer, fkNeedRetry, intSerde);
    +
    +            flush(producer,
    +                "data",
    +                stringSerde.serializer().serialize("", "flush"),
    +                intSerde.serializer().serialize("", 0)
    +            );
    +            flush(producer,
    +                "fk",
    +                intSerde.serializer().serialize("", 0),
    +                stringSerde.serializer().serialize("", "flush")
    +            );
    +        }
    +        return Collections.unmodifiableMap(allData);
    +    }
    +
    +    private static void retry(final KafkaProducer producer,
    +        List> needRetry,
    +        final Serde keySerde) {
    +        int remainingRetries = 5;
    +        while (!needRetry.isEmpty()) {
    +            final List> needRetry2 = new ArrayList<>();
    +            for (final ProducerRecord record : needRetry) {
    +                System.out.println(
    +                    "retry producing " + keySerde.deserializer().deserialize("", record.key()));
    +                producer.send(record, new TestCallback(record, needRetry2));
    +            }
    +            producer.flush();
    +            needRetry = needRetry2;
    +            if (--remainingRetries == 0 && !needRetry.isEmpty()) {
    +                System.err.println("Failed to produce all records after multiple retries");
    +                Exit.exit(1);
    +            }
    +        }
    +    }
    +
    +    private static void flush(final KafkaProducer producer,
    +        final String topic,
    +        final byte[] keyBytes,
    +        final byte[] valBytes) {
    +        // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out
    +        // all suppressed records.
    +        final List partitions = producer.partitionsFor(topic);
    +        for (final PartitionInfo partition : partitions) {
    +            producer.send(new ProducerRecord<>(
    +                partition.topic(),
    +                partition.partition(),
    +                System.currentTimeMillis() + Duration.ofDays(2).toMillis(),
    +                keyBytes,
    +                valBytes
    +            ));
    +        }
    +    }
    +
    +    private static Properties generatorProperties(final String kafka) {
    +        final Properties producerProps = new Properties();
    +        producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest");
    +        producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
    +        producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class);
    +        producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class);
    +        producerProps.put(ProducerConfig.ACKS_CONFIG, "all");
    +        return producerProps;
    +    }
    +
    +    private static class TestCallback implements Callback {
    +        private final ProducerRecord originalRecord;
    +        private final List> needRetry;
    +
    +        TestCallback(final ProducerRecord originalRecord,
    +                     final List> needRetry) {
    +            this.originalRecord = originalRecord;
    +            this.needRetry = needRetry;
    +        }
    +
    +        @Override
    +        public void onCompletion(final RecordMetadata metadata, final Exception exception) {
    +            if (exception != null) {
    +                if (exception instanceof TimeoutException) {
    +                    needRetry.add(originalRecord);
    +                } else {
    +                    exception.printStackTrace();
    +                    Exit.exit(1);
    +                }
    +            }
    +        }
    +    }
    +
    +    private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) {
    +        final Random rand = new Random();
    +        for (int i = 0; i < data.length; i++) {
    +            // we shuffle data within windowSize
    +            final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i;
    +
    +            // swap
    +            final int tmp = data[i];
    +            data[i] = data[j];
    +            data[j] = tmp;
    +        }
    +    }
    +
    +    public static class NumberDeserializer implements Deserializer {
    +        @Override
    +        public Number deserialize(final String topic, final byte[] data) {
    +            final Number value;
    +            switch (topic) {
    +                case "data":
    +                case "echo":
    +                case "min":
    +                case "min-raw":
    +                case "min-suppressed":
    +                case "sws-raw":
    +                case "sws-suppressed":
    +                case "max":
    +                case "dif":
    +                    value = intSerde.deserializer().deserialize(topic, data);
    +                    break;
    +                case "sum":
    +                case "cnt":
    +                case "tagg":
    +                    value = longSerde.deserializer().deserialize(topic, data);
    +                    break;
    +                case "avg":
    +                    value = doubleSerde.deserializer().deserialize(topic, data);
    +                    break;
    +                default:
    +                    throw new RuntimeException("unknown topic: " + topic);
    +            }
    +            return value;
    +        }
    +    }
    +
    +    public static VerificationResult verify(final String kafka,
    +                                            final Map> inputs,
    +                                            final int maxRecordsPerKey) {
    +        final Properties props = new Properties();
    +        props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier");
    +        props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
    +        props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
    +        props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class);
    +        props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed");
    +
    +        final KafkaConsumer consumer = new KafkaConsumer<>(props);
    +        final List partitions = getAllPartitions(consumer, NUMERIC_VALUE_TOPICS);
    +        consumer.assign(partitions);
    +        consumer.seekToBeginning(partitions);
    +
    +        final int recordsGenerated = inputs.size() * maxRecordsPerKey;
    +        int recordsProcessed = 0;
    +        final Map processed =
    +            Stream.of(NUMERIC_VALUE_TOPICS)
    +                  .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0)));
    +
    +        final Map>>> events = new HashMap<>();
    +
    +        VerificationResult verificationResult = new VerificationResult(false, "no results yet");
    +        int retry = 0;
    +        final long start = System.currentTimeMillis();
    +        while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) {
    +            final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5));
    +            if (records.isEmpty() && recordsProcessed >= recordsGenerated) {
    +                verificationResult = verifyAll(inputs, events, false);
    +                if (verificationResult.passed()) {
    +                    break;
    +                } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) {
    +                    System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries.");
    +                    break;
    +                } else {
    +                    System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry);
    +                }
    +            } else {
    +                System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry.");
    +
    +                retry = 0;
    +                for (final ConsumerRecord record : records) {
    +                    final String key = record.key();
    +
    +                    final String topic = record.topic();
    +                    processed.get(topic).incrementAndGet();
    +
    +                    if (topic.equals("echo")) {
    +                        recordsProcessed++;
    +                        if (recordsProcessed % 100 == 0) {
    +                            System.out.println("Echo records processed = " + recordsProcessed);
    +                        }
    +                    }
    +
    +                    events.computeIfAbsent(topic, t -> new HashMap<>())
    +                          .computeIfAbsent(key, k -> new LinkedList<>())
    +                          .add(record);
    +                }
    +
    +                System.out.println(processed);
    +            }
    +        }
    +        consumer.close();
    +        final long finished = System.currentTimeMillis() - start;
    +        System.out.println("Verification time=" + finished);
    +        System.out.println("-------------------");
    +        System.out.println("Result Verification");
    +        System.out.println("-------------------");
    +        System.out.println("recordGenerated=" + recordsGenerated);
    +        System.out.println("recordProcessed=" + recordsProcessed);
    +
    +        if (recordsProcessed > recordsGenerated) {
    +            System.out.println("PROCESSED-MORE-THAN-GENERATED");
    +        } else if (recordsProcessed < recordsGenerated) {
    +            System.out.println("PROCESSED-LESS-THAN-GENERATED");
    +        }
    +
    +        boolean success;
    +
    +        final Map> received =
    +            events.get("echo")
    +                  .entrySet()
    +                  .stream()
    +                  .map(entry -> mkEntry(
    +                      entry.getKey(),
    +                      entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet()))
    +                  )
    +                  .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
    +
    +        success = inputs.equals(received);
    +
    +        if (success) {
    +            System.out.println("ALL-RECORDS-DELIVERED");
    +        } else {
    +            int missedCount = 0;
    +            for (final Map.Entry> entry : inputs.entrySet()) {
    +                missedCount += received.get(entry.getKey()).size();
    +            }
    +            System.out.println("missedRecords=" + missedCount);
    +        }
    +
    +        // give it one more try if it's not already passing.
    +        if (!verificationResult.passed()) {
    +            verificationResult = verifyAll(inputs, events, true);
    +        }
    +        success &= verificationResult.passed();
    +
    +        System.out.println(verificationResult.result());
    +
    +        System.out.println(success ? "SUCCESS" : "FAILURE");
    +        return verificationResult;
    +    }
    +
    +    public static class VerificationResult {
    +        private final boolean passed;
    +        private final String result;
    +
    +        VerificationResult(final boolean passed, final String result) {
    +            this.passed = passed;
    +            this.result = result;
    +        }
    +
    +        public boolean passed() {
    +            return passed;
    +        }
    +
    +        public String result() {
    +            return result;
    +        }
    +    }
    +
    +    private static VerificationResult verifyAll(final Map> inputs,
    +                                                final Map>>> events,
    +                                                final boolean printResults) {
    +        final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
    +        boolean pass;
    +        try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) {
    +            pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults);
    +            pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults);
    +            pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> {
    +                final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", "");
    +                return getMin(unwindowedKey);
    +            }, printResults);
    +            pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults);
    +            pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults);
    +            pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults);
    +            pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults);
    +            pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults);
    +            pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults);
    +            pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults);
    +        }
    +        return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8));
    +    }
    +
    +    private static boolean verify(final PrintStream resultStream,
    +                                  final String topic,
    +                                  final Map> inputData,
    +                                  final Map>>> events,
    +                                  final Function keyToExpectation,
    +                                  final boolean printResults) {
    +        final Map>> observedInputEvents = events.get("data");
    +        final Map>> outputEvents = events.getOrDefault(topic, emptyMap());
    +        if (outputEvents.isEmpty()) {
    +            resultStream.println(topic + " is empty");
    +            return false;
    +        } else {
    +            resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size());
    +
    +            if (outputEvents.size() != inputData.size()) {
    +                resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n",
    +                                    outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet());
    +                return false;
    +            }
    +            for (final Map.Entry>> entry : outputEvents.entrySet()) {
    +                final String key = entry.getKey();
    +                final Number expected = keyToExpectation.apply(key);
    +                final Number actual = entry.getValue().getLast().value();
    +                if (!expected.equals(actual)) {
    +                    resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected);
    +
    +                    if (printResults) {
    +                        resultStream.printf("\t inputEvents=%n%s%n\t" +
    +                                "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n",
    +                            indent("\t\t", observedInputEvents.get(key)),
    +                            indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())),
    +                            indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())),
    +                            indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())),
    +                            indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())),
    +                            indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())),
    +                            indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>())));
    +
    +                        if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic))
    +                            resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue()));
    +                    }
    +
    +                    return false;
    +                }
    +            }
    +            return true;
    +        }
    +    }
    +
    +
    +    private static boolean verifySuppressed(final PrintStream resultStream,
    +                                            @SuppressWarnings("SameParameterValue") final String topic,
    +                                            final Map>>> events,
    +                                            final boolean printResults) {
    +        resultStream.println("verifying suppressed " + topic);
    +        final Map>> topicEvents = events.getOrDefault(topic, emptyMap());
    +        for (final Map.Entry>> entry : topicEvents.entrySet()) {
    +            if (entry.getValue().size() != 1) {
    +                final String unsuppressedTopic = topic.replace("-suppressed", "-raw");
    +                final String key = entry.getKey();
    +                final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", "");
    +                resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n",
    +                                    key,
    +                                    indent("\t\t", entry.getValue()));
    +
    +                if (printResults)
    +                    resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n",
    +                        indent("\t\t", events.get(unsuppressedTopic).get(key)),
    +                        indent("\t\t", events.get("data").get(unwindowedKey)));
    +
    +                return false;
    +            }
    +        }
    +        return true;
    +    }
    +
    +    private static String indent(@SuppressWarnings("SameParameterValue") final String prefix,
    +                                 final Iterable> list) {
    +        final StringBuilder stringBuilder = new StringBuilder();
    +        for (final ConsumerRecord record : list) {
    +            stringBuilder.append(prefix).append(record).append('\n');
    +        }
    +        return stringBuilder.toString();
    +    }
    +
    +    private static Long getSum(final String key) {
    +        final int min = getMin(key).intValue();
    +        final int max = getMax(key).intValue();
    +        return ((long) min + max) * (max - min + 1L) / 2L;
    +    }
    +
    +    private static Double getAvg(final String key) {
    +        final int min = getMin(key).intValue();
    +        final int max = getMax(key).intValue();
    +        return ((long) min + max) / 2.0;
    +    }
    +
    +
    +    private static boolean verifyTAgg(final PrintStream resultStream,
    +                                      final Map> allData,
    +                                      final Map>> taggEvents,
    +                                      final boolean printResults) {
    +        if (taggEvents == null) {
    +            resultStream.println("tagg is missing");
    +            return false;
    +        } else if (taggEvents.isEmpty()) {
    +            resultStream.println("tagg is empty");
    +            return false;
    +        } else {
    +            resultStream.println("verifying tagg");
    +
    +            // generate expected answer
    +            final Map expected = new HashMap<>();
    +            for (final String key : allData.keySet()) {
    +                final int min = getMin(key).intValue();
    +                final int max = getMax(key).intValue();
    +                final String cnt = Long.toString(max - min + 1L);
    +
    +                expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1);
    +            }
    +
    +            // check the result
    +            for (final Map.Entry>> entry : taggEvents.entrySet()) {
    +                final String key = entry.getKey();
    +                Long expectedCount = expected.remove(key);
    +                if (expectedCount == null) {
    +                    expectedCount = 0L;
    +                }
    +
    +                if (entry.getValue().getLast().value().longValue() != expectedCount) {
    +                    resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount);
    +
    +                    if (printResults)
    +                        resultStream.println("\t taggEvents: " + entry.getValue());
    +                    return false;
    +                }
    +            }
    +
    +        }
    +        return true;
    +    }
    +
    +    private static Number getMin(final String key) {
    +        return Integer.parseInt(key.split("-")[0]);
    +    }
    +
    +    private static Number getMax(final String key) {
    +        return Integer.parseInt(key.split("-")[1]);
    +    }
    +
    +    private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) {
    +        final List partitions = new ArrayList<>();
    +
    +        for (final String topic : topics) {
    +            for (final PartitionInfo info : consumer.partitionsFor(topic)) {
    +                partitions.add(new TopicPartition(info.topic(), info.partition()));
    +            }
    +        }
    +        return partitions;
    +    }
    +
    +}
    diff --git a/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java
    new file mode 100644
    index 0000000000..7cb34056e0
    --- /dev/null
    +++ b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java
    @@ -0,0 +1,131 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements. See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License. You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF 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.tests;
    +
    +import org.apache.kafka.common.serialization.Serde;
    +import org.apache.kafka.common.serialization.Serdes;
    +import org.apache.kafka.streams.KeyValue;
    +import org.apache.kafka.streams.kstream.Aggregator;
    +import org.apache.kafka.streams.kstream.Initializer;
    +import org.apache.kafka.streams.kstream.KeyValueMapper;
    +import org.apache.kafka.streams.kstream.Windowed;
    +import org.apache.kafka.streams.processor.api.ContextualProcessor;
    +import org.apache.kafka.streams.processor.api.ProcessorContext;
    +import org.apache.kafka.streams.processor.api.ProcessorSupplier;
    +import org.apache.kafka.streams.processor.api.Record;
    +
    +import java.time.Instant;
    +
    +public class SmokeTestUtil {
    +
    +    final static int END = Integer.MAX_VALUE;
    +
    +    static ProcessorSupplier printProcessorSupplier(final String topic) {
    +        return printProcessorSupplier(topic, "");
    +    }
    +
    +    static ProcessorSupplier printProcessorSupplier(final String topic, final String name) {
    +        return () -> new ContextualProcessor() {
    +            private int numRecordsProcessed = 0;
    +            private long smallestOffset = Long.MAX_VALUE;
    +            private long largestOffset = Long.MIN_VALUE;
    +
    +            @Override
    +            public void init(final ProcessorContext context) {
    +                super.init(context);
    +                System.out.println("[3.7] initializing processor: topic=" + topic + " taskId=" + context.taskId());
    +                System.out.flush();
    +                numRecordsProcessed = 0;
    +                smallestOffset = Long.MAX_VALUE;
    +                largestOffset = Long.MIN_VALUE;
    +            }
    +
    +            @Override
    +            public void process(final Record record) {
    +                numRecordsProcessed++;
    +                if (numRecordsProcessed % 100 == 0) {
    +                    System.out.printf("%s: %s%n", name, Instant.now());
    +                    System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic);
    +                }
    +
    +                context().recordMetadata().ifPresent(recordMetadata -> {
    +                    if (smallestOffset > recordMetadata.offset()) {
    +                        smallestOffset = recordMetadata.offset();
    +                    }
    +                    if (largestOffset < recordMetadata.offset()) {
    +                        largestOffset = recordMetadata.offset();
    +                    }
    +                });
    +            }
    +
    +            @Override
    +            public void close() {
    +                System.out.printf("Close processor for task %s%n", context().taskId());
    +                System.out.println("processed " + numRecordsProcessed + " records");
    +                final long processed;
    +                if (largestOffset >= smallestOffset) {
    +                    processed = 1L + largestOffset - smallestOffset;
    +                } else {
    +                    processed = 0L;
    +                }
    +                System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed);
    +                System.out.flush();
    +            }
    +        };
    +    }
    +
    +    public static final class Unwindow implements KeyValueMapper, V, K> {
    +        @Override
    +        public K apply(final Windowed winKey, final V value) {
    +            return winKey.key();
    +        }
    +    }
    +
    +    public static class Agg {
    +
    +        KeyValueMapper> selector() {
    +            return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L);
    +        }
    +
    +        public Initializer init() {
    +            return () -> 0L;
    +        }
    +
    +        Aggregator adder() {
    +            return (aggKey, value, aggregate) -> aggregate + value;
    +        }
    +
    +        Aggregator remover() {
    +            return (aggKey, value, aggregate) -> aggregate - value;
    +        }
    +    }
    +
    +    public static Serde stringSerde = Serdes.String();
    +
    +    public static Serde intSerde = Serdes.Integer();
    +
    +    static Serde longSerde = Serdes.Long();
    +
    +    static Serde doubleSerde = Serdes.Double();
    +
    +    public static void sleep(final long duration) {
    +        try {
    +            Thread.sleep(duration);
    +        } catch (final Exception ignore) { }
    +    }
    +
    +}
    diff --git a/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java
    new file mode 100644
    index 0000000000..5803b2fbd0
    --- /dev/null
    +++ b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.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.streams.tests;
    +
    +import org.apache.kafka.common.utils.Exit;
    +import org.apache.kafka.common.utils.Utils;
    +import org.apache.kafka.streams.StreamsConfig;
    +
    +import java.io.IOException;
    +import java.time.Duration;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +
    +import static org.apache.kafka.streams.tests.SmokeTestDriver.generate;
    +import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually;
    +
    +public class StreamsSmokeTest {
    +
    +    /**
    +     *  args ::= kafka propFileName command disableAutoTerminate
    +     *  command := "run" | "process"
    +     *
    +     * @param args
    +     */
    +    public static void main(final String[] args) throws IOException {
    +        if (args.length < 2) {
    +            System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter");
    +            Exit.exit(1);
    +        }
    +
    +        final String propFileName = args[0];
    +        final String command = args[1];
    +        final boolean disableAutoTerminate = args.length > 2;
    +
    +        final Properties streamsProperties = Utils.loadProps(propFileName);
    +        final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG);
    +        final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG);
    +
    +        if (kafka == null) {
    +            System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG);
    +            Exit.exit(1);
    +        }
    +
    +        if ("process".equals(command)) {
    +            if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) &&
    +                !StreamsConfig.EXACTLY_ONCE_V2.equals(processingGuarantee)) {
    +
    +                System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " +
    +                    StreamsConfig.EXACTLY_ONCE_V2);
    +
    +                Exit.exit(1);
    +            }
    +        }
    +
    +        System.out.println("StreamsTest instance started (StreamsSmokeTest)");
    +        System.out.println("command=" + command);
    +        System.out.println("props=" + streamsProperties);
    +        System.out.println("disableAutoTerminate=" + disableAutoTerminate);
    +
    +        switch (command) {
    +            case "run":
    +                // this starts the driver (data generation and result verification)
    +                final int numKeys = 10;
    +                final int maxRecordsPerKey = 500;
    +                if (disableAutoTerminate) {
    +                    generatePerpetually(kafka, numKeys, maxRecordsPerKey);
    +                } else {
    +                    // slow down data production to span 30 seconds so that system tests have time to
    +                    // do their bounces, etc.
    +                    final Map> allData =
    +                        generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30));
    +                    SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey);
    +                }
    +                break;
    +            case "process":
    +                // this starts the stream processing app
    +                new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties);
    +                break;
    +            default:
    +                System.out.println("unknown command: " + command);
    +        }
    +    }
    +
    +}
    diff --git a/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
    new file mode 100644
    index 0000000000..15769bf16c
    --- /dev/null
    +++ b/streams/upgrade-system-tests-37/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
    @@ -0,0 +1,120 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements. See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License. You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.kafka.streams.tests;
    +
    +import org.apache.kafka.common.utils.Utils;
    +import org.apache.kafka.streams.KafkaStreams;
    +import org.apache.kafka.streams.StreamsBuilder;
    +import org.apache.kafka.streams.StreamsConfig;
    +import org.apache.kafka.streams.kstream.Consumed;
    +import org.apache.kafka.streams.kstream.KStream;
    +import org.apache.kafka.streams.kstream.KTable;
    +import org.apache.kafka.streams.kstream.Produced;
    +import org.apache.kafka.streams.processor.api.ContextualProcessor;
    +import org.apache.kafka.streams.processor.api.ProcessorContext;
    +import org.apache.kafka.streams.processor.api.ProcessorSupplier;
    +import org.apache.kafka.streams.processor.api.Record;
    +
    +import java.util.Properties;
    +
    +import static org.apache.kafka.streams.tests.SmokeTestUtil.intSerde;
    +import static org.apache.kafka.streams.tests.SmokeTestUtil.stringSerde;
    +
    +
    +public class StreamsUpgradeTest {
    +
    +    @SuppressWarnings("unchecked")
    +    public static void main(final String[] args) throws Exception {
    +        if (args.length < 1) {
    +            System.err.println("StreamsUpgradeTest requires one argument (properties-file) but provided none");
    +        }
    +        final String propFileName = args[0];
    +
    +        final Properties streamsProperties = Utils.loadProps(propFileName);
    +
    +        System.out.println("StreamsTest instance started (StreamsUpgradeTest v3.7)");
    +        System.out.println("props=" + streamsProperties);
    +
    +        final StreamsBuilder builder = new StreamsBuilder();
    +        final KTable dataTable = builder.table(
    +            "data", Consumed.with(stringSerde, intSerde));
    +        final KStream dataStream = dataTable.toStream();
    +        dataStream.process(printProcessorSupplier("data"));
    +        dataStream.to("echo");
    +
    +        final boolean runFkJoin = Boolean.parseBoolean(streamsProperties.getProperty(
    +            "test.run_fk_join",
    +            "false"));
    +        if (runFkJoin) {
    +            try {
    +                final KTable fkTable = builder.table(
    +                    "fk", Consumed.with(intSerde, stringSerde));
    +                buildFKTable(dataStream, fkTable);
    +            } catch (final Exception e) {
    +                System.err.println("Caught " + e.getMessage());
    +            }
    +        }
    +
    +        final Properties config = new Properties();
    +        config.setProperty(
    +            StreamsConfig.APPLICATION_ID_CONFIG,
    +            "StreamsUpgradeTest");
    +        config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
    +        config.putAll(streamsProperties);
    +
    +        final KafkaStreams streams = new KafkaStreams(builder.build(), config);
    +        streams.start();
    +
    +        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
    +            streams.close();
    +            System.out.println("UPGRADE-TEST-CLIENT-CLOSED");
    +            System.out.flush();
    +        }));
    +    }
    +
    +    private static void buildFKTable(final KStream primaryTable,
    +                                     final KTable otherTable) {
    +        final KStream kStream = primaryTable.toTable()
    +            .join(otherTable, v -> v, (k0, v0) -> v0)
    +            .toStream();
    +        kStream.process(printProcessorSupplier("fk"));
    +        kStream.to("fk-result", Produced.with(stringSerde, stringSerde));
    +    }
    +
    +    private static  ProcessorSupplier printProcessorSupplier(final String topic) {
    +        return () -> new ContextualProcessor() {
    +            private int numRecordsProcessed = 0;
    +
    +            @Override
    +            public void init(final ProcessorContext context) {
    +                System.out.println("[3.7] initializing processor: topic=" + topic + "taskId=" + context.taskId());
    +                numRecordsProcessed = 0;
    +            }
    +
    +            @Override
    +            public void process(final Record record) {
    +                numRecordsProcessed++;
    +                if (numRecordsProcessed % 100 == 0) {
    +                    System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic);
    +                }
    +            }
    +
    +            @Override
    +            public void close() {}
    +        };
    +    }
    +}
    diff --git a/tests/kafkatest/services/kafka/consumer_group.py b/tests/kafkatest/services/kafka/consumer_group.py
    new file mode 100644
    index 0000000000..e94bd6382d
    --- /dev/null
    +++ b/tests/kafkatest/services/kafka/consumer_group.py
    @@ -0,0 +1,42 @@
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +
    +
    +# These are the group protocols we support. Most tests that use the new group coordinator will
    +# (eventually) be upgraded to test both of these consumer groups.
    +classic_group_protocol = 'classic'
    +consumer_group_protocol = 'consumer'
    +all_group_protocols = [classic_group_protocol, consumer_group_protocol]
    +
    +# These are the remote assignors used by the new group coordinator.
    +range_remote_assignor = 'range'
    +uniform_remote_assignor = 'uniform'
    +all_remote_assignors = [range_remote_assignor, uniform_remote_assignor]
    +
    +
    +def is_consumer_group_protocol_enabled(group_protocol):
    +    """Check if the KIP-848 consumer group protocol is enabled."""
    +    return group_protocol is not None and group_protocol.lower() == consumer_group_protocol
    +
    +
    +def maybe_set_group_protocol(group_protocol, config=None):
    +    """Maybe include the KIP-848 group.protocol configuration if it's not None."""
    +    if config is None:
    +        config = {}
    +
    +    if group_protocol is not None:
    +        config["group.protocol"] = group_protocol
    +
    +    return config
    diff --git a/tests/kafkatest/services/verifiable_consumer.py b/tests/kafkatest/services/verifiable_consumer.py
    index e1155c16aa..7ef5f75e22 100644
    --- a/tests/kafkatest/services/verifiable_consumer.py
    +++ b/tests/kafkatest/services/verifiable_consumer.py
    @@ -177,6 +177,8 @@ def __init__(self, context, num_nodes, kafka, topic, group_id,
             self.log_level = log_level
             self.kafka = kafka
             self.topic = topic
    +        self.group_protocol = group_protocol
    +        self.group_remote_assignor = group_remote_assignor
             self.group_id = group_id
             self.reset_policy = reset_policy
             self.static_membership = static_membership
    diff --git a/tests/kafkatest/tests/client/client_compatibility_features_test.py b/tests/kafkatest/tests/client/client_compatibility_features_test.py
    index 763d273738..5e31b99520 100644
    --- a/tests/kafkatest/tests/client/client_compatibility_features_test.py
    +++ b/tests/kafkatest/tests/client/client_compatibility_features_test.py
    @@ -19,7 +19,7 @@
     import time
     from random import randint
     
    -from ducktape.mark import matrix, parametrize
    +from ducktape.mark import matrix, parametrize, ignore
     from ducktape.mark.resource import cluster
     from ducktape.tests.test import TestContext
     
    @@ -28,8 +28,8 @@
     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, LATEST_1_0, \
         LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, \
    -    LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, V_0_11_0_0, \
    -    V_0_10_1_0, KafkaVersion
    +    LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, \
    +    V_0_11_0_0, V_0_10_1_0, KafkaVersion
     
     def get_broker_features(broker_version):
         features = {}
    @@ -116,30 +116,32 @@ def invoke_compatibility_program(self, features):
               self.logger.info("** Command failed.  See %s for log messages." % ssh_log_file)
               raise
     
    +    @ignore
         @cluster(num_nodes=7)
         @matrix(broker_version=[str(DEV_BRANCH)], metadata_quorum=quorum.all_non_upgrade)
    -    # @parametrize(broker_version=str(LATEST_0_10_0))
    -    # @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))
    -    # @parametrize(broker_version=str(LATEST_2_0))
    -    # @parametrize(broker_version=str(LATEST_2_1))
    -    # @parametrize(broker_version=str(LATEST_2_2))
    -    # @parametrize(broker_version=str(LATEST_2_3))
    -    # @parametrize(broker_version=str(LATEST_2_4))
    -    # @parametrize(broker_version=str(LATEST_2_5))
    -    # @parametrize(broker_version=str(LATEST_2_6))
    -    # @parametrize(broker_version=str(LATEST_2_7))
    -    # @parametrize(broker_version=str(LATEST_2_8))
    -    # @parametrize(broker_version=str(LATEST_3_0))
    -    # @parametrize(broker_version=str(LATEST_3_1))
    -    # @parametrize(broker_version=str(LATEST_3_2))
    -    # @parametrize(broker_version=str(LATEST_3_3))
    -    # @parametrize(broker_version=str(LATEST_3_4))
    -    # @parametrize(broker_version=str(LATEST_3_5))
    -    # @parametrize(broker_version=str(LATEST_3_6))
    +    @parametrize(broker_version=str(LATEST_0_10_0))
    +    @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))
    +    @parametrize(broker_version=str(LATEST_2_0))
    +    @parametrize(broker_version=str(LATEST_2_1))
    +    @parametrize(broker_version=str(LATEST_2_2))
    +    @parametrize(broker_version=str(LATEST_2_3))
    +    @parametrize(broker_version=str(LATEST_2_4))
    +    @parametrize(broker_version=str(LATEST_2_5))
    +    @parametrize(broker_version=str(LATEST_2_6))
    +    @parametrize(broker_version=str(LATEST_2_7))
    +    @parametrize(broker_version=str(LATEST_2_8))
    +    @parametrize(broker_version=str(LATEST_3_0))
    +    @parametrize(broker_version=str(LATEST_3_1))
    +    @parametrize(broker_version=str(LATEST_3_2))
    +    @parametrize(broker_version=str(LATEST_3_3))
    +    @parametrize(broker_version=str(LATEST_3_4))
    +    @parametrize(broker_version=str(LATEST_3_5))
    +    @parametrize(broker_version=str(LATEST_3_6))
    +    @parametrize(broker_version=str(LATEST_3_7))
         def run_compatibility_test(self, broker_version, metadata_quorum=quorum.zk):
             if self.zk:
                 self.zk.start()
    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 52ae86da43..28074d573c 100644
    --- a/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py
    +++ b/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py
    @@ -13,7 +13,7 @@
     # See the License for the specific language governing permissions and
     # limitations under the License.
     
    -from ducktape.mark import matrix, parametrize
    +from ducktape.mark import matrix, parametrize, ignore
     from ducktape.mark.resource import cluster
     from ducktape.utils.util import wait_until
     
    @@ -25,7 +25,8 @@
     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, LATEST_1_0, \
         LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, \
    -    LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, KafkaVersion
    +    LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, \
    +    KafkaVersion
     
     class ClientCompatibilityProduceConsumeTest(ProduceConsumeValidateTest):
         """
    @@ -56,30 +57,32 @@ def min_cluster_size(self):
             # Override this since we're adding services outside of the constructor
             return super(ClientCompatibilityProduceConsumeTest, self).min_cluster_size() + self.num_producers + self.num_consumers
     
    +    @ignore
         @cluster(num_nodes=9)
         @matrix(broker_version=[str(DEV_BRANCH)], metadata_quorum=quorum.all_non_upgrade)
    -    # @parametrize(broker_version=str(LATEST_0_10_0))
    -    # @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))
    -    # @parametrize(broker_version=str(LATEST_2_0))
    -    # @parametrize(broker_version=str(LATEST_2_1))
    -    # @parametrize(broker_version=str(LATEST_2_2))
    -    # @parametrize(broker_version=str(LATEST_2_3))
    -    # @parametrize(broker_version=str(LATEST_2_4))
    -    # @parametrize(broker_version=str(LATEST_2_5))
    -    # @parametrize(broker_version=str(LATEST_2_6))
    -    # @parametrize(broker_version=str(LATEST_2_7))
    -    # @parametrize(broker_version=str(LATEST_2_8))
    -    # @parametrize(broker_version=str(LATEST_3_0))
    -    # @parametrize(broker_version=str(LATEST_3_1))
    -    # @parametrize(broker_version=str(LATEST_3_2))
    -    # @parametrize(broker_version=str(LATEST_3_3))
    -    # @parametrize(broker_version=str(LATEST_3_4))
    -    # @parametrize(broker_version=str(LATEST_3_5))
    -    # @parametrize(broker_version=str(LATEST_3_6))
    +    @parametrize(broker_version=str(LATEST_0_10_0))
    +    @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))
    +    @parametrize(broker_version=str(LATEST_2_0))
    +    @parametrize(broker_version=str(LATEST_2_1))
    +    @parametrize(broker_version=str(LATEST_2_2))
    +    @parametrize(broker_version=str(LATEST_2_3))
    +    @parametrize(broker_version=str(LATEST_2_4))
    +    @parametrize(broker_version=str(LATEST_2_5))
    +    @parametrize(broker_version=str(LATEST_2_6))
    +    @parametrize(broker_version=str(LATEST_2_7))
    +    @parametrize(broker_version=str(LATEST_2_8))
    +    @parametrize(broker_version=str(LATEST_3_0))
    +    @parametrize(broker_version=str(LATEST_3_1))
    +    @parametrize(broker_version=str(LATEST_3_2))
    +    @parametrize(broker_version=str(LATEST_3_3))
    +    @parametrize(broker_version=str(LATEST_3_4))
    +    @parametrize(broker_version=str(LATEST_3_5))
    +    @parametrize(broker_version=str(LATEST_3_6))
    +    @parametrize(broker_version=str(LATEST_3_7))
         def test_produce_consume(self, broker_version, metadata_quorum=quorum.zk):
             print("running producer_consumer_compat with broker_version = %s" % broker_version, flush=True)
             self.kafka.set_version(KafkaVersion(broker_version))
    diff --git a/tests/kafkatest/tests/client/consumer_rolling_upgrade_test.py b/tests/kafkatest/tests/client/consumer_rolling_upgrade_test.py
    index c60e18460c..9f1db343a7 100644
    --- a/tests/kafkatest/tests/client/consumer_rolling_upgrade_test.py
    +++ b/tests/kafkatest/tests/client/consumer_rolling_upgrade_test.py
    @@ -18,7 +18,7 @@
     
     
     from kafkatest.tests.verifiable_consumer_test import VerifiableConsumerTest
    -from kafkatest.services.kafka import TopicPartition, quorum
    +from kafkatest.services.kafka import TopicPartition, quorum, consumer_group
     
     class ConsumerRollingUpgradeTest(VerifiableConsumerTest):
         TOPIC = "test_topic"
    @@ -56,7 +56,12 @@ def _verify_roundrobin_assignment(self, consumer):
             metadata_quorum=[quorum.isolated_kraft],
             use_new_coordinator=[True, False]
         )
    -    def rolling_update_test(self, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    @matrix(
    +        metadata_quorum=quorum.all_kraft,
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
    +    )
    +    def rolling_update_test(self, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             """
             Verify rolling updates of partition assignment strategies works correctly. In this
             test, we use a rolling restart to change the group's assignment strategy from "range" 
    @@ -65,7 +70,7 @@ def rolling_update_test(self, metadata_quorum=quorum.zk, use_new_coordinator=Fal
             """
     
             # initialize the consumer using range assignment
    -        consumer = self.setup_consumer(self.TOPIC, assignment_strategy=self.RANGE)
    +        consumer = self.setup_consumer(self.TOPIC, assignment_strategy=self.RANGE, group_protocol=group_protocol)
     
             consumer.start()
             self.await_all_members(consumer)
    diff --git a/tests/kafkatest/tests/client/consumer_test.py b/tests/kafkatest/tests/client/consumer_test.py
    index 2e8ad38f6f..0436f16c78 100644
    --- a/tests/kafkatest/tests/client/consumer_test.py
    +++ b/tests/kafkatest/tests/client/consumer_test.py
    @@ -18,7 +18,7 @@
     from ducktape.mark.resource import cluster
     
     from kafkatest.tests.verifiable_consumer_test import VerifiableConsumerTest
    -from kafkatest.services.kafka import TopicPartition, quorum
    +from kafkatest.services.kafka import TopicPartition, quorum, consumer_group
     
     import signal
     
    @@ -75,15 +75,16 @@ def setup_consumer(self, topic, **kwargs):
             return consumer
     
         @cluster(num_nodes=7)
    -    # @matrix(
    -    #     metadata_quorum=[quorum.zk],
    -    #     use_new_coordinator=[False]
    -    # )
         @matrix(
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
    +    )
    +    @matrix(
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
         )
    -    def test_broker_rolling_bounce(self, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    def test_broker_rolling_bounce(self, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             """
             Verify correct consumer behavior when the brokers are consecutively restarted.
     
    @@ -108,7 +109,7 @@ def test_broker_rolling_bounce(self, metadata_quorum=quorum.zk, use_new_coordina
             # broker, and that coordinator will fail the consumer and trigger a group rebalance if its session times out.
             # This test is asserting that no rebalances occur, so we increase the session timeout for this to be the case.
             self.session_timeout_sec = 30
    -        consumer = self.setup_consumer(self.TOPIC)
    +        consumer = self.setup_consumer(self.TOPIC, group_protocol=group_protocol)
     
             producer.start()
             self.await_produced_messages(producer)
    @@ -133,19 +134,20 @@ def test_broker_rolling_bounce(self, metadata_quorum=quorum.zk, use_new_coordina
                 (consumer.total_consumed(), consumer.current_position(partition))
     
         @cluster(num_nodes=7)
    -    # @matrix(
    -    #     clean_shutdown=[True],
    -    #     bounce_mode=["all", "rolling"],
    -    #     metadata_quorum=[quorum.zk],
    -    #     use_new_coordinator=[False]
    -    # )
         @matrix(
             clean_shutdown=[True],
             bounce_mode=["all", "rolling"],
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]   
    +        use_new_coordinator=[False]
         )
    -    def test_consumer_bounce(self, clean_shutdown, bounce_mode, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    @matrix(
    +        clean_shutdown=[True],
    +        bounce_mode=["all", "rolling"],
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
    +    )
    +    def test_consumer_bounce(self, clean_shutdown, bounce_mode, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             """
             Verify correct consumer behavior when the consumers in the group are consecutively restarted.
     
    @@ -160,7 +162,7 @@ def test_consumer_bounce(self, clean_shutdown, bounce_mode, metadata_quorum=quor
             partition = TopicPartition(self.TOPIC, 0)
     
             producer = self.setup_producer(self.TOPIC)
    -        consumer = self.setup_consumer(self.TOPIC)
    +        consumer = self.setup_consumer(self.TOPIC, group_protocol=group_protocol)
     
             producer.start()
             self.await_produced_messages(producer)
    @@ -368,22 +370,23 @@ def test_fencing_static_consumer(self, num_conflict_consumers, fencing_stage, me
                            )
     
         @cluster(num_nodes=7)
    -    # @matrix(
    -    #     clean_shutdown=[True],
    -    #     enable_autocommit=[True, False],
    -    #     metadata_quorum=[quorum.zk],
    -    #     use_new_coordinator=[False]
    -    # )
         @matrix(
             clean_shutdown=[True],
             enable_autocommit=[True, False],
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
    +    )
    +    @matrix(
    +        clean_shutdown=[True],
    +        enable_autocommit=[True, False],
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
         )
    -    def test_consumer_failure(self, clean_shutdown, enable_autocommit, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    def test_consumer_failure(self, clean_shutdown, enable_autocommit, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             partition = TopicPartition(self.TOPIC, 0)
     
    -        consumer = self.setup_consumer(self.TOPIC, enable_autocommit=enable_autocommit)
    +        consumer = self.setup_consumer(self.TOPIC, enable_autocommit=enable_autocommit, group_protocol=group_protocol)
             producer = self.setup_producer(self.TOPIC)
     
             consumer.start()
    @@ -436,12 +439,19 @@ def test_consumer_failure(self, clean_shutdown, enable_autocommit, metadata_quor
             clean_shutdown=[True],
             enable_autocommit=[True, False],
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True]
    +        use_new_coordinator=[False]
    +    )
    +    @matrix(
    +        clean_shutdown=[True, False],
    +        enable_autocommit=[True, False],
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
         )
    -    def test_broker_failure(self, clean_shutdown, enable_autocommit, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    def test_broker_failure(self, clean_shutdown, enable_autocommit, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             partition = TopicPartition(self.TOPIC, 0)
     
    -        consumer = self.setup_consumer(self.TOPIC, enable_autocommit=enable_autocommit)
    +        consumer = self.setup_consumer(self.TOPIC, enable_autocommit=enable_autocommit, group_protocol=group_protocol)
             producer = self.setup_producer(self.TOPIC)
     
             producer.start()
    @@ -474,15 +484,16 @@ def test_broker_failure(self, clean_shutdown, enable_autocommit, metadata_quorum
                     (consumer.last_commit(partition), consumer.current_position(partition))
     
         @cluster(num_nodes=7)
    -    # @matrix(
    -    #     metadata_quorum=[quorum.zk],
    -    #     use_new_coordinator=[False]
    -    # )
         @matrix(
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
    +    )
    +    @matrix(
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
         )
    -    def test_group_consumption(self, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    def test_group_consumption(self, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             """
             Verifies correct group rebalance behavior as consumers are started and stopped.
             In particular, this test verifies that the partition is readable after every
    @@ -494,7 +505,7 @@ def test_group_consumption(self, metadata_quorum=quorum.zk, use_new_coordinator=
             - Start the consumers one by one, verifying consumption after each rebalance
             - Shutdown the consumers one by one, verifying consumption after each rebalance
             """
    -        consumer = self.setup_consumer(self.TOPIC)
    +        consumer = self.setup_consumer(self.TOPIC, group_protocol=group_protocol)
             producer = self.setup_producer(self.TOPIC)
     
             partition = TopicPartition(self.TOPIC, 0)
    @@ -532,21 +543,28 @@ def __init__(self, test_context):
             })
     
         @cluster(num_nodes=6)
    -    # @matrix(
    -    #     assignment_strategy=["org.apache.kafka.clients.consumer.RangeAssignor",
    -    #                          "org.apache.kafka.clients.consumer.RoundRobinAssignor",
    -    #                          "org.apache.kafka.clients.consumer.StickyAssignor"],
    -    #     metadata_quorum=[quorum.zk],
    -    #     use_new_coordinator=[False]
    -    # )
         @matrix(
             assignment_strategy=["org.apache.kafka.clients.consumer.RangeAssignor",
                                  "org.apache.kafka.clients.consumer.RoundRobinAssignor",
    -                             "org.apache.kafka.clients.consumer.StickyAssignor"], 
    +                             "org.apache.kafka.clients.consumer.StickyAssignor"],
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
    +    )
    +    @matrix(
    +        assignment_strategy=["org.apache.kafka.clients.consumer.RangeAssignor",
    +                             "org.apache.kafka.clients.consumer.RoundRobinAssignor",
    +                             "org.apache.kafka.clients.consumer.StickyAssignor"],
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=[consumer_group.classic_group_protocol],
    +    )
    +    @matrix(
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=[consumer_group.consumer_group_protocol],
    +        group_remote_assignor=consumer_group.all_remote_assignors
         )
    -    def test_valid_assignment(self, assignment_strategy, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    def test_valid_assignment(self, assignment_strategy=None, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None, group_remote_assignor=None):
             """
             Verify assignment strategy correctness: each partition is assigned to exactly
             one consumer instance.
    @@ -556,7 +574,10 @@ def test_valid_assignment(self, assignment_strategy, metadata_quorum=quorum.zk,
             - Start the consumers one by one
             - Validate assignment after every expected rebalance
             """
    -        consumer = self.setup_consumer(self.TOPIC, assignment_strategy=assignment_strategy)
    +        consumer = self.setup_consumer(self.TOPIC,
    +                                       assignment_strategy=assignment_strategy,
    +                                       group_protocol=group_protocol,
    +                                       group_remote_assignor=group_remote_assignor)
             for num_started, node in enumerate(consumer.nodes, 1):
                 consumer.start_node(node)
                 self.await_members(consumer, num_started)
    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 de06c32141..e88114d87c 100644
    --- a/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py
    +++ b/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py
    @@ -24,7 +24,7 @@
     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, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, \
         LATEST_2_7, LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, \
    -    DEV_BRANCH, KafkaVersion
    +    LATEST_3_7, 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):
    @@ -64,6 +64,7 @@ def setUp(self):
         @matrix(producer_version=[str(LATEST_3_4)], consumer_version=[str(LATEST_3_4)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade)
         @matrix(producer_version=[str(LATEST_3_5)], consumer_version=[str(LATEST_3_5)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade)
         @matrix(producer_version=[str(LATEST_3_6)], consumer_version=[str(LATEST_3_6)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade)
    +    @matrix(producer_version=[str(LATEST_3_7)], consumer_version=[str(LATEST_3_7)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade)
         @matrix(producer_version=[str(LATEST_2_1)], consumer_version=[str(LATEST_2_1)], compression_types=[["zstd"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade)
         @matrix(producer_version=[str(LATEST_2_0)], consumer_version=[str(LATEST_2_0)], compression_types=[["snappy"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade)
         @matrix(producer_version=[str(LATEST_1_1)], consumer_version=[str(LATEST_1_1)], compression_types=[["lz4"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade)
    diff --git a/tests/kafkatest/tests/core/consume_bench_test.py b/tests/kafkatest/tests/core/consume_bench_test.py
    index 6304569959..e8e6334d00 100644
    --- a/tests/kafkatest/tests/core/consume_bench_test.py
    +++ b/tests/kafkatest/tests/core/consume_bench_test.py
    @@ -17,7 +17,7 @@
     from ducktape.mark import matrix
     from ducktape.mark.resource import cluster
     from ducktape.tests.test import Test
    -from kafkatest.services.kafka import KafkaService, quorum
    +from kafkatest.services.kafka import KafkaService, quorum, consumer_group
     from kafkatest.services.trogdor.produce_bench_workload import ProduceBenchWorkloadService, ProduceBenchWorkloadSpec
     from kafkatest.services.trogdor.consume_bench_workload import ConsumeBenchWorkloadService, ConsumeBenchWorkloadSpec
     from kafkatest.services.trogdor.task_spec import TaskSpec
    @@ -82,9 +82,18 @@ def produce_messages(self, topics, max_messages=10000):
                 ["consume_bench_topic[0-5]:[0-4]"] # manual topic assignment
             ],
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
         )
    -    def test_consume_bench(self, topics, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    @matrix(
    +        topics=[
    +            ["consume_bench_topic[0-5]"], # topic subscription
    +            ["consume_bench_topic[0-5]:[0-4]"] # manual topic assignment
    +        ],
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
    +    )
    +    def test_consume_bench(self, topics, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             """
             Runs a ConsumeBench workload to consume messages
             """
    @@ -94,7 +103,7 @@ def test_consume_bench(self, topics, metadata_quorum=quorum.zk, use_new_coordina
                                                     self.consumer_workload_service.bootstrap_servers,
                                                     target_messages_per_sec=1000,
                                                     max_messages=10000,
    -                                                consumer_conf={},
    +                                                consumer_conf=consumer_group.maybe_set_group_protocol(group_protocol),
                                                     admin_client_conf={},
                                                     common_client_conf={},
                                                     active_topics=topics)
    @@ -111,9 +120,14 @@ def test_consume_bench(self, topics, metadata_quorum=quorum.zk, use_new_coordina
         # )
         @matrix(
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
         )
    -    def test_single_partition(self, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    @matrix(
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
    +    )
    +    def test_single_partition(self, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             """
             Run a ConsumeBench against a single partition
             """
    @@ -124,7 +138,7 @@ def test_single_partition(self, metadata_quorum=quorum.zk, use_new_coordinator=F
                                                     self.consumer_workload_service.bootstrap_servers,
                                                     target_messages_per_sec=1000,
                                                     max_messages=2500,
    -                                                consumer_conf={},
    +                                                consumer_conf=consumer_group.maybe_set_group_protocol(group_protocol),
                                                     admin_client_conf={},
                                                     common_client_conf={},
                                                     active_topics=["consume_bench_topic:1"])
    @@ -141,9 +155,14 @@ def test_single_partition(self, metadata_quorum=quorum.zk, use_new_coordinator=F
         # )
         @matrix(
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
    +    )
    +    @matrix(
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
         )
    -    def test_multiple_consumers_random_group_topics(self, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    def test_multiple_consumers_random_group_topics(self, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             """
             Runs multiple consumers group to read messages from topics.
             Since a consumerGroup isn't specified, each consumer should read from all topics independently
    @@ -154,7 +173,7 @@ def test_multiple_consumers_random_group_topics(self, metadata_quorum=quorum.zk,
                                                     self.consumer_workload_service.bootstrap_servers,
                                                     target_messages_per_sec=1000,
                                                     max_messages=5000, # all should read exactly 5k messages
    -                                                consumer_conf={},
    +                                                consumer_conf=consumer_group.maybe_set_group_protocol(group_protocol),
                                                     admin_client_conf={},
                                                     common_client_conf={},
                                                     threads_per_worker=5,
    @@ -172,9 +191,14 @@ def test_multiple_consumers_random_group_topics(self, metadata_quorum=quorum.zk,
         # )
         @matrix(
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
    +    )
    +    @matrix(
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
         )
    -    def test_two_consumers_specified_group_topics(self, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    def test_two_consumers_specified_group_topics(self, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             """
             Runs two consumers in the same consumer group to read messages from topics.
             Since a consumerGroup is specified, each consumer should dynamically get assigned a partition from group
    @@ -185,7 +209,7 @@ def test_two_consumers_specified_group_topics(self, metadata_quorum=quorum.zk, u
                                                     self.consumer_workload_service.bootstrap_servers,
                                                     target_messages_per_sec=1000,
                                                     max_messages=2000, # both should read at least 2k messages
    -                                                consumer_conf={},
    +                                                consumer_conf=consumer_group.maybe_set_group_protocol(group_protocol),
                                                     admin_client_conf={},
                                                     common_client_conf={},
                                                     threads_per_worker=2,
    @@ -204,9 +228,14 @@ def test_two_consumers_specified_group_topics(self, metadata_quorum=quorum.zk, u
         # )
         @matrix(
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
         )
    -    def test_multiple_consumers_random_group_partitions(self, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    @matrix(
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
    +    )
    +    def test_multiple_consumers_random_group_partitions(self, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             """
             Runs multiple consumers in to read messages from specific partitions.
             Since a consumerGroup isn't specified, each consumer will get assigned a random group
    @@ -218,7 +247,7 @@ def test_multiple_consumers_random_group_partitions(self, metadata_quorum=quorum
                                                     self.consumer_workload_service.bootstrap_servers,
                                                     target_messages_per_sec=1000,
                                                     max_messages=2000,
    -                                                consumer_conf={},
    +                                                consumer_conf=consumer_group.maybe_set_group_protocol(group_protocol),
                                                     admin_client_conf={},
                                                     common_client_conf={},
                                                     threads_per_worker=4,
    @@ -236,9 +265,14 @@ def test_multiple_consumers_random_group_partitions(self, metadata_quorum=quorum
         # )
         @matrix(
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
    +    )
    +    @matrix(
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
         )
    -    def test_multiple_consumers_specified_group_partitions_should_raise(self, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    def test_multiple_consumers_specified_group_partitions_should_raise(self, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             """
             Runs multiple consumers in the same group to read messages from specific partitions.
             It is an invalid configuration to provide a consumer group and specific partitions.
    @@ -250,7 +284,7 @@ def test_multiple_consumers_specified_group_partitions_should_raise(self, metada
                                                     self.consumer_workload_service.bootstrap_servers,
                                                     target_messages_per_sec=1000,
                                                     max_messages=2000,
    -                                                consumer_conf={},
    +                                                consumer_conf=consumer_group.maybe_set_group_protocol(group_protocol),
                                                     admin_client_conf={},
                                                     common_client_conf={},
                                                     threads_per_worker=4,
    diff --git a/tests/kafkatest/tests/core/consumer_group_command_test.py b/tests/kafkatest/tests/core/consumer_group_command_test.py
    index f7c0f26d99..2df53e3093 100644
    --- a/tests/kafkatest/tests/core/consumer_group_command_test.py
    +++ b/tests/kafkatest/tests/core/consumer_group_command_test.py
    @@ -20,7 +20,7 @@
     from ducktape.mark.resource import cluster
     
     from kafkatest.services.zookeeper import ZookeeperService
    -from kafkatest.services.kafka import KafkaService, quorum
    +from kafkatest.services.kafka import KafkaService, quorum, consumer_group
     from kafkatest.services.console_consumer import ConsoleConsumer
     from kafkatest.services.security.security_config import SecurityConfig
     
    @@ -59,14 +59,15 @@ def start_kafka(self, security_protocol, interbroker_security_protocol):
                 controller_num_nodes_override=self.num_zk)
             self.kafka.start()
     
    -    def start_consumer(self):
    +    def start_consumer(self, group_protocol=None):
    +        consumer_properties = consumer_group.maybe_set_group_protocol(group_protocol)
             self.consumer = ConsoleConsumer(self.test_context, num_nodes=self.num_brokers, kafka=self.kafka, topic=TOPIC,
    -                                        consumer_timeout_ms=None)
    +                                        consumer_timeout_ms=None, consumer_properties=consumer_properties)
             self.consumer.start()
     
    -    def setup_and_verify(self, security_protocol, group=None):
    +    def setup_and_verify(self, security_protocol, group=None, group_protocol=None):
             self.start_kafka(security_protocol, security_protocol)
    -        self.start_consumer()
    +        self.start_consumer(group_protocol=group_protocol)
             consumer_node = self.consumer.nodes[0]
             wait_until(lambda: self.consumer.alive(consumer_node),
                        timeout_sec=20, backoff_sec=.2, err_msg="Consumer was too slow to start")
    @@ -90,37 +91,39 @@ def setup_and_verify(self, security_protocol, group=None):
             self.consumer.stop()
     
         @cluster(num_nodes=3)
    -    # @matrix(
    -    #     security_protocol=['PLAINTEXT', 'SSL'],
    -    #     metadata_quorum=[quorum.zk],
    -    #     use_new_coordinator=[False]
    -    # )
         @matrix(
             security_protocol=['PLAINTEXT', 'SSL'],
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
         )
    -    def test_list_consumer_groups(self, security_protocol='PLAINTEXT', metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    @matrix(
    +        security_protocol=['PLAINTEXT', 'SSL'],
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
    +    )
    +    def test_list_consumer_groups(self, security_protocol='PLAINTEXT', metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             """
             Tests if ConsumerGroupCommand is listing correct consumer groups
             :return: None
             """
    -        self.setup_and_verify(security_protocol)
    +        self.setup_and_verify(security_protocol, group_protocol=group_protocol)
     
         @cluster(num_nodes=3)
    -    # @matrix(
    -    #     security_protocol=['PLAINTEXT', 'SSL'],
    -    #     metadata_quorum=[quorum.zk],
    -    #     use_new_coordinator=[False]
    -    # )
         @matrix(
             security_protocol=['PLAINTEXT', 'SSL'],
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
    +    )
    +    @matrix(
    +        security_protocol=['PLAINTEXT', 'SSL'],
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
         )
    -    def test_describe_consumer_group(self, security_protocol='PLAINTEXT', metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    def test_describe_consumer_group(self, security_protocol='PLAINTEXT', metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             """
             Tests if ConsumerGroupCommand is describing a consumer group correctly
             :return: None
             """
    -        self.setup_and_verify(security_protocol, group="test-consumer-group")
    +        self.setup_and_verify(security_protocol, group="test-consumer-group", group_protocol=group_protocol)
    diff --git a/tests/kafkatest/tests/core/downgrade_test.py b/tests/kafkatest/tests/core/downgrade_test.py
    index fede8d5c2f..404d251ceb 100644
    --- a/tests/kafkatest/tests/core/downgrade_test.py
    +++ b/tests/kafkatest/tests/core/downgrade_test.py
    @@ -21,7 +21,7 @@
     from kafkatest.tests.end_to_end import EndToEndTest
     from kafkatest.version import LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, \
         LATEST_2_6, LATEST_2_7, LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, \
    -    LATEST_3_6, DEV_BRANCH, KafkaVersion
    +    LATEST_3_6, LATEST_3_7, DEV_BRANCH, KafkaVersion
     
     class TestDowngrade(EndToEndTest):
         PARTITIONS = 3
    @@ -82,6 +82,9 @@ def wait_until_rejoin(self):
                         timeout_sec=60, backoff_sec=1, err_msg="Replicas did not rejoin the ISR in a reasonable amount of time")
     
         @cluster(num_nodes=7)
    +    @parametrize(version=str(LATEST_3_7), compression_types=["snappy"])
    +    @parametrize(version=str(LATEST_3_7), compression_types=["zstd"], security_protocol="SASL_SSL")
    +    @matrix(version=[str(LATEST_3_7)], compression_types=[["none"]], static_membership=[False, True])
         @parametrize(version=str(LATEST_3_6), compression_types=["snappy"])
         @parametrize(version=str(LATEST_3_6), compression_types=["zstd"], security_protocol="SASL_SSL")
         @matrix(version=[str(LATEST_3_6)], compression_types=[["none"]], static_membership=[False, True])
    diff --git a/tests/kafkatest/tests/core/fetch_from_follower_test.py b/tests/kafkatest/tests/core/fetch_from_follower_test.py
    index dd69b60399..a4c810116d 100644
    --- a/tests/kafkatest/tests/core/fetch_from_follower_test.py
    +++ b/tests/kafkatest/tests/core/fetch_from_follower_test.py
    @@ -20,7 +20,7 @@
     from ducktape.mark.resource import cluster
     
     from kafkatest.services.console_consumer import ConsoleConsumer
    -from kafkatest.services.kafka import KafkaService, quorum
    +from kafkatest.services.kafka import KafkaService, quorum, consumer_group
     from kafkatest.services.monitor.jmx import JmxTool
     from kafkatest.services.verifiable_producer import VerifiableProducer
     from kafkatest.services.zookeeper import ZookeeperService
    @@ -70,15 +70,16 @@ def setUp(self):
             self.kafka.start()
     
         @cluster(num_nodes=9)
    -    # @matrix(
    -    #     metadata_quorum=[quorum.zk],
    -    #     use_new_coordinator=[False]
    -    # )
         @matrix(
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
         )
    -    def test_consumer_preferred_read_replica(self, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    @matrix(
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
    +    )
    +    def test_consumer_preferred_read_replica(self, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             """
             This test starts up brokers with "broker.rack" and "replica.selector.class" configurations set. The replica
             selector is set to the rack-aware implementation. One of the brokers has a different rack than the other two.
    @@ -98,10 +99,15 @@ def test_consumer_preferred_read_replica(self, metadata_quorum=quorum.zk, use_ne
     
             self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic,
                                                throughput=self.producer_throughput)
    +        consumer_properties = consumer_group.maybe_set_group_protocol(group_protocol,
    +                                                                      config={
    +                                                                          "client.rack": non_leader_rack,
    +                                                                          "metadata.max.age.ms": self.METADATA_MAX_AGE_MS
    +                                                                      })
             self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic,
                                             client_id="console-consumer", group_id="test-consumer-group-1",
                                             consumer_timeout_ms=60000, message_validator=is_int,
    -                                        consumer_properties={"client.rack": non_leader_rack, "metadata.max.age.ms": self.METADATA_MAX_AGE_MS})
    +                                        consumer_properties=consumer_properties)
     
             # Start up and let some data get produced
             self.start_producer_and_consumer()
    diff --git a/tests/kafkatest/tests/core/kraft_upgrade_test.py b/tests/kafkatest/tests/core/kraft_upgrade_test.py
    index f6a2918695..3f3c4a81b1 100644
    --- a/tests/kafkatest/tests/core/kraft_upgrade_test.py
    +++ b/tests/kafkatest/tests/core/kraft_upgrade_test.py
    @@ -23,7 +23,7 @@
     from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
     from kafkatest.utils import is_int
     from kafkatest.version import LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, \
    -    DEV_BRANCH, KafkaVersion, LATEST_METADATA_VERSION
    +    DEV_BRANCH, KafkaVersion, LATEST_STABLE_METADATA_VERSION
     
     #
     # Test upgrading between different KRaft versions.
    @@ -71,8 +71,8 @@ def perform_version_change(self, from_kafka_version):
                 self.kafka.start_node(node)
                 self.wait_until_rejoin()
                 self.logger.info("Successfully restarted broker node %s" % node.account.hostname)
    -        self.logger.info("Changing metadata.version to %s" % LATEST_METADATA_VERSION)
    -        self.kafka.upgrade_metadata_version(LATEST_METADATA_VERSION)
    +        self.logger.info("Changing metadata.version to %s" % LATEST_STABLE_METADATA_VERSION)
    +        self.kafka.upgrade_metadata_version(LATEST_STABLE_METADATA_VERSION)
     
         def run_upgrade(self, from_kafka_version):
             """Test upgrade of Kafka broker cluster from various versions to the current version
    diff --git a/tests/kafkatest/tests/core/reassign_partitions_test.py b/tests/kafkatest/tests/core/reassign_partitions_test.py
    index d908462996..2ff698c9ab 100644
    --- a/tests/kafkatest/tests/core/reassign_partitions_test.py
    +++ b/tests/kafkatest/tests/core/reassign_partitions_test.py
    @@ -19,7 +19,7 @@
     
     from kafkatest.services.kafka import config_property
     from kafkatest.services.zookeeper import ZookeeperService
    -from kafkatest.services.kafka import KafkaService, quorum
    +from kafkatest.services.kafka import KafkaService, quorum, consumer_group
     from kafkatest.services.verifiable_producer import VerifiableProducer
     from kafkatest.services.console_consumer import ConsoleConsumer
     from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
    @@ -135,19 +135,20 @@ def move_start_offset(self):
             time.sleep(12 + 30)
     
         @cluster(num_nodes=8)
    -    # @matrix(
    -    #     bounce_brokers=[True, False],
    -    #     reassign_from_offset_zero=[True, False],
    -    #     metadata_quorum=[quorum.zk],
    -    #     use_new_coordinator=[False]
    -    # )
         @matrix(
             bounce_brokers=[True, False],
             reassign_from_offset_zero=[True, False],
             metadata_quorum=[quorum.isolated_kraft],
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
         )
    -    def test_reassign_partitions(self, bounce_brokers, reassign_from_offset_zero, metadata_quorum, use_new_coordinator=False):
    +    @matrix(
    +        bounce_brokers=[True, False],
    +        reassign_from_offset_zero=[True, False],
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
    +    )
    +    def test_reassign_partitions(self, bounce_brokers, reassign_from_offset_zero, metadata_quorum, use_new_coordinator=False, group_protocol=None):
             """Reassign partitions tests.
             Setup: 1 zk, 4 kafka nodes, 1 topic with partitions=20, replication-factor=3,
             and min.insync.replicas=3
    @@ -170,7 +171,8 @@ def test_reassign_partitions(self, bounce_brokers, reassign_from_offset_zero, me
             self.consumer = ConsoleConsumer(self.test_context, self.num_consumers,
                                             self.kafka, self.topic,
                                             consumer_timeout_ms=60000,
    -                                        message_validator=is_int)
    +                                        message_validator=is_int,
    +                                        consumer_properties=consumer_group.maybe_set_group_protocol(group_protocol))
     
             self.enable_idempotence=True
             self.run_produce_consume_validate(core_test_action=lambda: self.reassign_partitions(bounce_brokers))
    diff --git a/tests/kafkatest/tests/core/replica_scale_test.py b/tests/kafkatest/tests/core/replica_scale_test.py
    index 103cbe596f..9b837b8aed 100644
    --- a/tests/kafkatest/tests/core/replica_scale_test.py
    +++ b/tests/kafkatest/tests/core/replica_scale_test.py
    @@ -20,7 +20,7 @@
     from kafkatest.services.trogdor.produce_bench_workload import ProduceBenchWorkloadService, ProduceBenchWorkloadSpec
     from kafkatest.services.trogdor.consume_bench_workload import ConsumeBenchWorkloadService, ConsumeBenchWorkloadSpec
     from kafkatest.services.trogdor.task_spec import TaskSpec
    -from kafkatest.services.kafka import KafkaService, quorum
    +from kafkatest.services.kafka import KafkaService, quorum, consumer_group
     from kafkatest.services.trogdor.trogdor import TrogdorService
     from kafkatest.services.zookeeper import ZookeeperService
     
    @@ -63,8 +63,16 @@ def teardown(self):
             metadata_quorum=[quorum.isolated_kraft],
             use_new_coordinator=[True]
         )
    -    def test_produce_consume(self, topic_count, partition_count, replication_factor, 
    -                             metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    @matrix(
    +        topic_count=[3],
    +        partition_count=[34],
    +        replication_factor=[3],
    +        metadata_quorum=[quorum.isolated_kraft],
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
    +    )
    +    def test_produce_consume(self, topic_count, partition_count, replication_factor,
    +                             metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             topics_create_start_time = time.time()
             for i in range(topic_count):
                 topic = "replicas_produce_consume_%d" % i
    @@ -104,12 +112,13 @@ def test_produce_consume(self, topic_count, partition_count, replication_factor,
             produce_workload.wait_for_done(timeout_sec=600)
             print("Completed produce bench", flush=True)  # Force some stdout for Travis
     
    +        consumer_conf = consumer_group.maybe_set_group_protocol(group_protocol)
             consume_spec = ConsumeBenchWorkloadSpec(0, TaskSpec.MAX_DURATION_MS,
                                                     consumer_workload_service.consumer_node,
                                                     consumer_workload_service.bootstrap_servers,
                                                     target_messages_per_sec=150000,
                                                     max_messages=1700000,
    -                                                consumer_conf={},
    +                                                consumer_conf=consumer_conf,
                                                     admin_client_conf={},
                                                     common_client_conf={},
                                                     active_topics=["replicas_produce_consume_[0-2]"])
    diff --git a/tests/kafkatest/tests/core/snapshot_test.py b/tests/kafkatest/tests/core/snapshot_test.py
    index 5368b48e5c..25ffd4227e 100644
    --- a/tests/kafkatest/tests/core/snapshot_test.py
    +++ b/tests/kafkatest/tests/core/snapshot_test.py
    @@ -21,6 +21,7 @@
     from kafkatest.services.console_consumer import ConsoleConsumer
     from kafkatest.services.kafka import KafkaService
     from kafkatest.services.kafka import config_property
    +from kafkatest.services.kafka import consumer_group
     from kafkatest.services.verifiable_producer import VerifiableProducer
     from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
     from kafkatest.utils import is_int
    @@ -125,7 +126,7 @@ def file_exists(self, node, file_path):
                 self.logger.debug("File %s was found" % file_path)
                 return True
     
    -    def validate_success(self, topic = None):
    +    def validate_success(self, topic = None, group_protocol=None):
             if topic is None:
                 # Create a new topic
                 topic = "%s%d" % (TestSnapshots.TOPIC_NAME_PREFIX, self.topics_created)
    @@ -138,7 +139,8 @@ def validate_success(self, topic = None):
     
             self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka,
                                             topic, consumer_timeout_ms=30000,
    -                                        message_validator=is_int)
    +                                        message_validator=is_int,
    +                                        consumer_properties=consumer_group.maybe_set_group_protocol(group_protocol))
             self.start_producer_and_consumer()
             self.stop_producer_and_consumer()
             self.validate()
    @@ -146,9 +148,14 @@ def validate_success(self, topic = None):
         @cluster(num_nodes=9)
         @matrix(
             metadata_quorum=quorum.all_kraft,
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
         )
    -    def test_broker(self, metadata_quorum=quorum.combined_kraft, use_new_coordinator=False):
    +    @matrix(
    +        metadata_quorum=quorum.all_kraft,
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
    +    )
    +    def test_broker(self, metadata_quorum=quorum.combined_kraft, use_new_coordinator=False, group_protocol=None):
             """ Test the ability of a broker to consume metadata snapshots
             and to recover the cluster metadata state using them
     
    @@ -204,14 +211,19 @@ def test_broker(self, metadata_quorum=quorum.combined_kraft, use_new_coordinator
             self.kafka.create_topic(topic_cfg)
     
             # Produce to the newly created topic and make sure it works.
    -        self.validate_success(broker_topic)
    +        self.validate_success(broker_topic, group_protocol=group_protocol)
     
         @cluster(num_nodes=9)
         @matrix(
             metadata_quorum=quorum.all_kraft,
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
    +    )
    +    @matrix(
    +        metadata_quorum=quorum.all_kraft,
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
         )
    -    def test_controller(self, metadata_quorum=quorum.combined_kraft, use_new_coordinator=False):
    +    def test_controller(self, metadata_quorum=quorum.combined_kraft, use_new_coordinator=False, group_protocol=None):
             """ Test the ability of controllers to consume metadata snapshots
             and to recover the cluster metadata state using them
     
    @@ -254,4 +266,4 @@ def test_controller(self, metadata_quorum=quorum.combined_kraft, use_new_coordin
                 self.kafka.controller_quorum.start_node(node)
     
             # Produce to a newly created topic and make sure it works.
    -        self.validate_success()
    +        self.validate_success(group_protocol=group_protocol)
    diff --git a/tests/kafkatest/tests/core/transactions_test.py b/tests/kafkatest/tests/core/transactions_test.py
    index ad89135341..efceba9607 100644
    --- a/tests/kafkatest/tests/core/transactions_test.py
    +++ b/tests/kafkatest/tests/core/transactions_test.py
    @@ -14,7 +14,7 @@
     # limitations under the License.
     
     from kafkatest.services.zookeeper import ZookeeperService
    -from kafkatest.services.kafka import KafkaService, quorum
    +from kafkatest.services.kafka import KafkaService, quorum, consumer_group
     from kafkatest.services.console_consumer import ConsoleConsumer
     from kafkatest.services.verifiable_producer import VerifiableProducer
     from kafkatest.services.transactional_message_copier import TransactionalMessageCopier
    @@ -85,8 +85,8 @@ def seed_messages(self, topic, num_seed_messages):
                        (self.num_seed_messages, seed_timeout_sec))
             return seed_producer.acked
     
    -    def get_messages_from_topic(self, topic, num_messages):
    -        consumer = self.start_consumer(topic, group_id="verifying_consumer")
    +    def get_messages_from_topic(self, topic, num_messages, group_protocol):
    +        consumer = self.start_consumer(topic, group_id="verifying_consumer", group_protocol=group_protocol)
             return self.drain_consumer(consumer, num_messages)
     
         def bounce_brokers(self, clean_shutdown):
    @@ -155,7 +155,7 @@ def create_and_start_copiers(self, input_topic, output_topic, num_copiers, use_g
                 ))
             return copiers
     
    -    def start_consumer(self, topic_to_read, group_id):
    +    def start_consumer(self, topic_to_read, group_id, group_protocol):
             consumer = ConsoleConsumer(context=self.test_context,
                                        num_nodes=1,
                                        kafka=self.kafka,
    @@ -163,7 +163,8 @@ def start_consumer(self, topic_to_read, group_id):
                                        group_id=group_id,
                                        message_validator=is_int,
                                        from_beginning=True,
    -                                   isolation_level="read_committed")
    +                                   isolation_level="read_committed",
    +                                   consumer_properties=consumer_group.maybe_set_group_protocol(group_protocol))
             consumer.start()
             # ensure that the consumer is up.
             wait_until(lambda: (len(consumer.messages_consumed[1]) > 0) == True,
    @@ -190,7 +191,7 @@ def drain_consumer(self, consumer, num_messages):
         def copy_messages_transactionally(self, failure_mode, bounce_target,
                                           input_topic, output_topic,
                                           num_copiers, num_messages_to_copy,
    -                                      use_group_metadata):
    +                                      use_group_metadata, group_protocol):
             """Copies messages transactionally from the seeded input topic to the
             output topic, either bouncing brokers or clients in a hard and soft
             way as it goes.
    @@ -205,7 +206,8 @@ def copy_messages_transactionally(self, failure_mode, bounce_target,
                                                     num_copiers=num_copiers,
                                                     use_group_metadata=use_group_metadata)
             concurrent_consumer = self.start_consumer(output_topic,
    -                                                  group_id="concurrent_consumer")
    +                                                  group_id="concurrent_consumer",
    +                                                  group_protocol=group_protocol)
             clean_shutdown = False
             if failure_mode == "clean_bounce":
                 clean_shutdown = True
    @@ -258,9 +260,18 @@ def setup_topics(self):
             check_order=[True, False],
             use_group_metadata=[True, False],
             metadata_quorum=quorum.all_kraft,
    -        use_new_coordinator=[True, False]
    +        use_new_coordinator=[False]
         )
    -    def test_transactions(self, failure_mode, bounce_target, check_order, use_group_metadata, metadata_quorum=quorum.zk, use_new_coordinator=False):
    +    @matrix(
    +        failure_mode=["hard_bounce", "clean_bounce"],
    +        bounce_target=["brokers", "clients"],
    +        check_order=[True, False],
    +        use_group_metadata=[True, False],
    +        metadata_quorum=quorum.all_kraft,
    +        use_new_coordinator=[True],
    +        group_protocol=consumer_group.all_group_protocols
    +    )
    +    def test_transactions(self, failure_mode, bounce_target, check_order, use_group_metadata, metadata_quorum=quorum.zk, use_new_coordinator=False, group_protocol=None):
             security_protocol = 'PLAINTEXT'
             self.kafka.security_protocol = security_protocol
             self.kafka.interbroker_security_protocol = security_protocol
    @@ -284,8 +295,8 @@ def test_transactions(self, failure_mode, bounce_target, check_order, use_group_
             concurrently_consumed_messages = self.copy_messages_transactionally(
                 failure_mode, bounce_target, input_topic=self.input_topic,
                 output_topic=self.output_topic, num_copiers=self.num_input_partitions,
    -            num_messages_to_copy=self.num_seed_messages, use_group_metadata=use_group_metadata)
    -        output_messages = self.get_messages_from_topic(self.output_topic, self.num_seed_messages)
    +            num_messages_to_copy=self.num_seed_messages, use_group_metadata=use_group_metadata, group_protocol=group_protocol)
    +        output_messages = self.get_messages_from_topic(self.output_topic, self.num_seed_messages, group_protocol)
     
             concurrently_consumed_message_set = set(concurrently_consumed_messages)
             output_message_set = set(output_messages)
    diff --git a/tests/kafkatest/tests/core/upgrade_test.py b/tests/kafkatest/tests/core/upgrade_test.py
    index 62855b78c8..8ba1d0a02d 100644
    --- a/tests/kafkatest/tests/core/upgrade_test.py
    +++ b/tests/kafkatest/tests/core/upgrade_test.py
    @@ -27,7 +27,7 @@
     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, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, \
         LATEST_2_6, LATEST_2_7, LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, \
    -    LATEST_3_6, V_0_11_0_0, V_2_8_0, V_3_0_0, DEV_BRANCH, KafkaVersion
    +    LATEST_3_6, LATEST_3_7, V_0_11_0_0, V_2_8_0, V_3_0_0, DEV_BRANCH, KafkaVersion
     from kafkatest.services.kafka.util import new_jdk_not_supported
     
     class TestUpgrade(ProduceConsumeValidateTest):
    @@ -94,6 +94,9 @@ def perform_upgrade(self, from_kafka_version, to_message_format_version=None):
                 self.wait_until_rejoin()
     
         @cluster(num_nodes=6)
    +    @parametrize(from_kafka_version=str(LATEST_3_7), to_message_format_version=None, compression_types=["none"])
    +    @parametrize(from_kafka_version=str(LATEST_3_7), to_message_format_version=None, compression_types=["lz4"])
    +    @parametrize(from_kafka_version=str(LATEST_3_7), to_message_format_version=None, compression_types=["snappy"])
         @parametrize(from_kafka_version=str(LATEST_3_6), to_message_format_version=None, compression_types=["none"])
         @parametrize(from_kafka_version=str(LATEST_3_6), to_message_format_version=None, compression_types=["lz4"])
         @parametrize(from_kafka_version=str(LATEST_3_6), to_message_format_version=None, compression_types=["snappy"])
    diff --git a/tests/kafkatest/tests/streams/streams_application_upgrade_test.py b/tests/kafkatest/tests/streams/streams_application_upgrade_test.py
    index d52ea57464..56c900c60f 100644
    --- a/tests/kafkatest/tests/streams/streams_application_upgrade_test.py
    +++ b/tests/kafkatest/tests/streams/streams_application_upgrade_test.py
    @@ -22,13 +22,13 @@
     from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService
     from kafkatest.services.zookeeper import ZookeeperService
     from kafkatest.version import LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \
    -  LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, DEV_VERSION, KafkaVersion
    +  LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, DEV_VERSION, KafkaVersion
     
     smoke_test_versions = [str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4),
                            str(LATEST_2_5), str(LATEST_2_6), str(LATEST_2_7),
                            str(LATEST_2_8), str(LATEST_3_0), str(LATEST_3_1),
                            str(LATEST_3_2), str(LATEST_3_3), str(LATEST_3_4),
    -                       str(LATEST_3_5), str(LATEST_3_6)]
    +                       str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7)]
     dev_version = [str(DEV_VERSION)]
     
     class StreamsUpgradeTest(Test):
    diff --git a/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py
    index e4c6220191..3e133c09ed 100644
    --- a/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py
    +++ b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py
    @@ -23,7 +23,7 @@
     from kafkatest.services.zookeeper import ZookeeperService
     from kafkatest.version import LATEST_0_11_0, LATEST_0_10_2, LATEST_0_10_1, LATEST_0_10_0, LATEST_1_0, LATEST_1_1, \
         LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \
    -    LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, KafkaVersion
    +    LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, KafkaVersion
     
     
     class StreamsBrokerCompatibility(Test):
    @@ -65,25 +65,11 @@ def setUp(self):
     
         @ignore # AutoMQ won't release stream jar, so we only need AutoMQ can support stream
         @cluster(num_nodes=4)
    -    @parametrize(broker_version=str(LATEST_3_6))
    -    @parametrize(broker_version=str(LATEST_3_5))
    -    @parametrize(broker_version=str(LATEST_3_4))
    -    @parametrize(broker_version=str(LATEST_3_3))
    -    @parametrize(broker_version=str(LATEST_3_2))
    -    @parametrize(broker_version=str(LATEST_3_1))
    -    @parametrize(broker_version=str(LATEST_3_0))
    -    @parametrize(broker_version=str(LATEST_2_8))
    -    @parametrize(broker_version=str(LATEST_2_7))
    -    @parametrize(broker_version=str(LATEST_2_6))
    -    @parametrize(broker_version=str(LATEST_2_5))
    -    @parametrize(broker_version=str(LATEST_2_4))
    -    @parametrize(broker_version=str(LATEST_2_3))
    -    @parametrize(broker_version=str(LATEST_2_2))
    -    @parametrize(broker_version=str(LATEST_2_1))
    -    @parametrize(broker_version=str(LATEST_2_0))
    -    @parametrize(broker_version=str(LATEST_1_1))
    -    @parametrize(broker_version=str(LATEST_1_0))
    -    @parametrize(broker_version=str(LATEST_0_11_0))
    +    @matrix(broker_version=[str(LATEST_0_11_0),str(LATEST_1_0),str(LATEST_1_1),str(LATEST_2_0),
    +                            str(LATEST_2_1),str(LATEST_2_2),str(LATEST_2_3),str(LATEST_2_4),
    +                            str(LATEST_2_5),str(LATEST_2_6),str(LATEST_2_7),str(LATEST_2_8),
    +                            str(LATEST_3_0),str(LATEST_3_1),str(LATEST_3_2),str(LATEST_3_3),
    +                            str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7)])
         def test_compatible_brokers_eos_disabled(self, broker_version):
             self.kafka.set_version(KafkaVersion(broker_version))
             self.kafka.start()
    @@ -102,25 +88,11 @@ def test_compatible_brokers_eos_disabled(self, broker_version):
     
         @ignore
         @cluster(num_nodes=4)
    -    @parametrize(broker_version=str(LATEST_3_6))
    -    @parametrize(broker_version=str(LATEST_3_5))
    -    @parametrize(broker_version=str(LATEST_3_4))
    -    @parametrize(broker_version=str(LATEST_3_3))
    -    @parametrize(broker_version=str(LATEST_3_2))
    -    @parametrize(broker_version=str(LATEST_3_1))
    -    @parametrize(broker_version=str(LATEST_3_0))
    -    @parametrize(broker_version=str(LATEST_2_8))
    -    @parametrize(broker_version=str(LATEST_2_7))
    -    @parametrize(broker_version=str(LATEST_2_6))
    -    @parametrize(broker_version=str(LATEST_2_5))
    -    @parametrize(broker_version=str(LATEST_2_4))
    -    @parametrize(broker_version=str(LATEST_2_3))
    -    @parametrize(broker_version=str(LATEST_2_2))
    -    @parametrize(broker_version=str(LATEST_2_1))
    -    @parametrize(broker_version=str(LATEST_2_0))
    -    @parametrize(broker_version=str(LATEST_1_1))
    -    @parametrize(broker_version=str(LATEST_1_0))
    -    @parametrize(broker_version=str(LATEST_0_11_0))
    +    @matrix(broker_version=[str(LATEST_0_11_0),str(LATEST_1_0),str(LATEST_1_1),str(LATEST_2_0),
    +                            str(LATEST_2_1),str(LATEST_2_2),str(LATEST_2_3),str(LATEST_2_4),
    +                            str(LATEST_2_5),str(LATEST_2_6),str(LATEST_2_7),str(LATEST_2_8),
    +                            str(LATEST_3_0),str(LATEST_3_1),str(LATEST_3_2),str(LATEST_3_3),
    +                            str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7)])
         def test_compatible_brokers_eos_alpha_enabled(self, broker_version):
             self.kafka.set_version(KafkaVersion(broker_version))
             self.kafka.start()
    @@ -139,17 +111,9 @@ def test_compatible_brokers_eos_alpha_enabled(self, broker_version):
     
         @ignore
         @cluster(num_nodes=4)
    -    @parametrize(broker_version=str(LATEST_3_6))
    -    @parametrize(broker_version=str(LATEST_3_5))
    -    @parametrize(broker_version=str(LATEST_3_4))
    -    @parametrize(broker_version=str(LATEST_3_3))
    -    @parametrize(broker_version=str(LATEST_3_2))
    -    @parametrize(broker_version=str(LATEST_3_1))
    -    @parametrize(broker_version=str(LATEST_3_0))
    -    @parametrize(broker_version=str(LATEST_2_8))
    -    @parametrize(broker_version=str(LATEST_2_7))
    -    @parametrize(broker_version=str(LATEST_2_6))
    -    @parametrize(broker_version=str(LATEST_2_5))
    +    @matrix(broker_version=[str(LATEST_2_5),str(LATEST_2_6),str(LATEST_2_7),str(LATEST_2_8),
    +                            str(LATEST_3_0),str(LATEST_3_1),str(LATEST_3_2),str(LATEST_3_3),
    +                            str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7)])
         def test_compatible_brokers_eos_v2_enabled(self, broker_version):
             self.kafka.set_version(KafkaVersion(broker_version))
             self.kafka.start()
    diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py
    index 1c5c54fa15..ddc0ffe2e4 100644
    --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py
    +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py
    @@ -26,7 +26,7 @@
     from kafkatest.tests.streams.utils import extract_generation_from_logs, extract_generation_id
     from kafkatest.version import LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, LATEST_1_0, LATEST_1_1, \
         LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \
    -    LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, DEV_BRANCH, DEV_VERSION, \
    +    LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, DEV_BRANCH, DEV_VERSION, \
         KafkaVersion
     
     # broker 0.10.0 is not compatible with newer Kafka Streams versions
    @@ -36,7 +36,7 @@
                                str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LATEST_2_7),
                                str(LATEST_2_8), str(LATEST_3_0), str(LATEST_3_1), str(LATEST_3_2),
                                str(LATEST_3_3), str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6),
    -                           str(DEV_BRANCH)]
    +                           str(LATEST_3_7), str(DEV_BRANCH)]
     
     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),
    @@ -46,7 +46,7 @@
     # -> https://issues.apache.org/jira/browse/KAFKA-14646
     # thus, we cannot test two bounce rolling upgrade because we know it's broken
     # instead we add version 2.4...3.3 to the `metadata_2_versions` upgrade list
    -fk_join_versions = [str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6)]
    +fk_join_versions = [str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7)]
     
     
     """
    diff --git a/tests/kafkatest/tests/verifiable_consumer_test.py b/tests/kafkatest/tests/verifiable_consumer_test.py
    index e84601c187..1a92b2c24c 100644
    --- a/tests/kafkatest/tests/verifiable_consumer_test.py
    +++ b/tests/kafkatest/tests/verifiable_consumer_test.py
    @@ -54,10 +54,11 @@ def min_cluster_size(self):
             return super(VerifiableConsumerTest, self).min_cluster_size() + self.num_consumers + self.num_producers
     
         def setup_consumer(self, topic, static_membership=False, enable_autocommit=False,
    -                       assignment_strategy="org.apache.kafka.clients.consumer.RangeAssignor", **kwargs):
    +                       assignment_strategy="org.apache.kafka.clients.consumer.RangeAssignor", group_remote_assignor="range", **kwargs):
             return VerifiableConsumer(self.test_context, self.num_consumers, self.kafka,
                                       topic, self.group_id, static_membership=static_membership, session_timeout_sec=self.session_timeout_sec,
                                       assignment_strategy=assignment_strategy, enable_autocommit=enable_autocommit,
    +                                  group_remote_assignor=group_remote_assignor,
                                       log_level="TRACE", **kwargs)
     
         def setup_producer(self, topic, max_messages=-1, throughput=500):
    diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py
    index 3e3da017bc..9f21c120fc 100644
    --- a/tests/kafkatest/version.py
    +++ b/tests/kafkatest/version.py
    @@ -124,7 +124,8 @@ def get_version(node=None):
     DEV_BRANCH = KafkaVersion("dev")
     DEV_VERSION = KafkaVersion("3.8.0-SNAPSHOT")
     
    -LATEST_METADATA_VERSION = "3.8"
    +# This should match the LATEST_PRODUCTION version defined in MetadataVersion.java
    +LATEST_STABLE_METADATA_VERSION = "3.7"
     
     # 0.8.2.x versions
     V_0_8_2_1 = KafkaVersion("0.8.2.1")
    diff --git a/tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java b/tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java
    index 5e44865b20..6bf82bedc5 100644
    --- a/tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java
    +++ b/tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java
    @@ -24,6 +24,7 @@
     import org.apache.kafka.clients.admin.DeleteRecordsResult;
     import org.apache.kafka.clients.admin.RecordsToDelete;
     import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.utils.Exit;
     import org.apache.kafka.common.utils.Utils;
     import org.apache.kafka.server.common.AdminCommandFailedException;
     import org.apache.kafka.server.common.AdminOperationException;
    @@ -61,7 +62,18 @@ public class DeleteRecordsCommand {
         private static final DecodeJson.DecodeString STRING = new DecodeJson.DecodeString();
     
         public static void main(String[] args) throws Exception {
    -        execute(args, System.out);
    +        Exit.exit(mainNoExit(args));
    +    }
    +
    +    static int mainNoExit(String... args) {
    +        try {
    +            execute(args, System.out);
    +            return 0;
    +        } catch (Exception e) {
    +            System.err.println(e.getMessage());
    +            System.err.println(Utils.stackTrace(e));
    +            return 1;
    +        }
         }
     
         static Map> parseOffsetJsonStringWithoutDedup(String jsonData) throws JsonProcessingException {
    diff --git a/tools/src/main/java/org/apache/kafka/tools/GetOffsetShell.java b/tools/src/main/java/org/apache/kafka/tools/GetOffsetShell.java
    index 99551ca054..0d90327bbe 100644
    --- a/tools/src/main/java/org/apache/kafka/tools/GetOffsetShell.java
    +++ b/tools/src/main/java/org/apache/kafka/tools/GetOffsetShell.java
    @@ -59,6 +59,7 @@
     import java.util.stream.Collectors;
     
     public class GetOffsetShell {
    +    static final String USAGE_TEXT = "An interactive shell for getting topic-partition offsets.";
         private static final Pattern TOPIC_PARTITION_PATTERN = Pattern.compile("([^:,]*)(?::(?:([0-9]*)|(?:([0-9]*)-([0-9]*))))?");
     
         public static void main(String... args) {
    @@ -142,7 +143,7 @@ public GetOffsetShellOptions(String[] args) throws TerseException {
                 excludeInternalTopicsOpt = parser.accepts("exclude-internal-topics", "By default, internal topics are included. If specified, internal topics are excluded.");
     
                 if (args.length == 0) {
    -                CommandLineUtils.printUsageAndExit(parser, "An interactive shell for getting topic-partition offsets.");
    +                CommandLineUtils.printUsageAndExit(parser, USAGE_TEXT);
                 }
     
                 try {
    @@ -157,6 +158,8 @@ public GetOffsetShellOptions(String[] args) throws TerseException {
                     effectiveBrokerListOpt = brokerListOpt;
                 }
     
    +            CommandLineUtils.maybePrintHelpOrVersion(this, USAGE_TEXT);
    +
                 CommandLineUtils.checkRequiredArgs(parser, options, effectiveBrokerListOpt);
     
                 String brokerList = options.valueOf(effectiveBrokerListOpt);
    diff --git a/tools/src/main/java/org/apache/kafka/tools/LeaderElectionCommand.java b/tools/src/main/java/org/apache/kafka/tools/LeaderElectionCommand.java
    index c704e41875..e1cdeed1e5 100644
    --- a/tools/src/main/java/org/apache/kafka/tools/LeaderElectionCommand.java
    +++ b/tools/src/main/java/org/apache/kafka/tools/LeaderElectionCommand.java
    @@ -28,6 +28,7 @@
     import org.apache.kafka.common.errors.ClusterAuthorizationException;
     import org.apache.kafka.common.errors.ElectionNotNeededException;
     import org.apache.kafka.common.errors.TimeoutException;
    +import org.apache.kafka.common.utils.Exit;
     import org.apache.kafka.common.utils.Utils;
     import org.apache.kafka.server.common.AdminCommandFailedException;
     import org.apache.kafka.server.common.AdminOperationException;
    @@ -62,11 +63,17 @@ public class LeaderElectionCommand {
         private static final DecodeJson.DecodeInteger INT = new DecodeJson.DecodeInteger();
     
         public static void main(String... args) {
    +        Exit.exit(mainNoExit(args));
    +    }
    +
    +    static int mainNoExit(String... args) {
             try {
                 run(Duration.ofMillis(30000), args);
    -        } catch (Exception e) {
    +            return 0;
    +        } catch (Throwable e) {
                 System.err.println(e.getMessage());
                 System.err.println(Utils.stackTrace(e));
    +            return 1;
             }
         }
     
    diff --git a/tools/src/main/java/org/apache/kafka/tools/ToolsUtils.java b/tools/src/main/java/org/apache/kafka/tools/ToolsUtils.java
    index 9c6a7a0d1c..394f5078c4 100644
    --- a/tools/src/main/java/org/apache/kafka/tools/ToolsUtils.java
    +++ b/tools/src/main/java/org/apache/kafka/tools/ToolsUtils.java
    @@ -16,9 +16,11 @@
      */
     package org.apache.kafka.tools;
     
    +import joptsimple.OptionParser;
     import org.apache.kafka.common.Metric;
     import org.apache.kafka.common.MetricName;
     import org.apache.kafka.common.utils.Utils;
    +import org.apache.kafka.server.util.CommandLineUtils;
     
     import java.io.PrintStream;
     import java.util.Arrays;
    @@ -155,4 +157,17 @@ public static  Set minus(Set set, T...toRemove) {
             return res;
         }
     
    +    /**
    +     * This is a simple wrapper around `CommandLineUtils.printUsageAndExit`.
    +     * It is needed for tools migration (KAFKA-14525), as there is no Java equivalent for return type `Nothing`.
    +     * Can be removed once [[kafka.tools.ConsoleConsumer]]
    +     * and [[kafka.tools.ConsoleProducer]] are migrated.
    +     *
    +     * @param parser Command line options parser.
    +     * @param message Error message.
    +     */
    +    public static void printUsageAndExit(OptionParser parser, String message) {
    +        CommandLineUtils.printUsageAndExit(parser, message);
    +        throw new AssertionError("printUsageAndExit should not return, but it did.");
    +    }
     }
    diff --git a/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java b/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java
    index 043ae521f5..66650cb9db 100644
    --- a/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java
    +++ b/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java
    @@ -208,9 +208,9 @@ private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionRea
          *                           If set to true, the command will throw an exception if the topic with the
          *                           requested name does not exist.
          */
    -    private static void ensureTopicExists(List foundTopics, String requestedTopic, Boolean requireTopicExists) {
    +    private static void ensureTopicExists(List foundTopics, Optional requestedTopic, Boolean requireTopicExists) {
             // If no topic name was mentioned, do not need to throw exception.
    -        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
    +        if (requestedTopic.isPresent() && !requestedTopic.get().isEmpty() && requireTopicExists && foundTopics.isEmpty()) {
                 // If given topic doesn't exist then throw exception
                 throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
             }
    @@ -490,7 +490,7 @@ public void listTopics(TopicCommandOptions opts) throws ExecutionException, Inte
             public void alterTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
                 CommandTopicPartition topic = new CommandTopicPartition(opts);
                 List topics = getTopics(opts.topic(), opts.excludeInternalTopics());
    -            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
    +            ensureTopicExists(topics, opts.topic(), !opts.ifExists());
     
                 if (!topics.isEmpty()) {
                     Map> topicsInfo = adminClient.describeTopics(topics).topicNameValues();
    @@ -556,7 +556,7 @@ public void describeTopic(TopicCommandOptions opts) throws ExecutionException, I
                 if (useTopicId) {
                     ensureTopicIdExists(topicIds, inputTopicId.get(), !opts.ifExists());
                 } else {
    -                ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
    +                ensureTopicExists(topics, opts.topic(), !opts.ifExists());
                 }
                 List topicDescriptions = new ArrayList<>();
     
    @@ -632,7 +632,7 @@ numPartitions, getReplicationFactor(firstPartition, reassignment),
     
             public void deleteTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
                 List topics = getTopics(opts.topic(), opts.excludeInternalTopics());
    -            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
    +            ensureTopicExists(topics, opts.topic(), !opts.ifExists());
                 adminClient.deleteTopics(Collections.unmodifiableList(topics),
                     new DeleteTopicsOptions().retryOnQuotaViolation(false)
                 ).all().get();
    diff --git a/tools/src/main/java/org/apache/kafka/tools/Tuple2.java b/tools/src/main/java/org/apache/kafka/tools/Tuple2.java
    deleted file mode 100644
    index 02dd4bf398..0000000000
    --- a/tools/src/main/java/org/apache/kafka/tools/Tuple2.java
    +++ /dev/null
    @@ -1,48 +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.tools;
    -
    -import java.util.Objects;
    -
    -public final class Tuple2 {
    -    public final V1 v1;
    -
    -    public final V2 v2;
    -
    -    public Tuple2(V1 v1, V2 v2) {
    -        this.v1 = v1;
    -        this.v2 = v2;
    -    }
    -
    -    @Override
    -    public boolean equals(Object o) {
    -        if (this == o) return true;
    -        if (o == null || getClass() != o.getClass()) return false;
    -        Tuple2 tuple = (Tuple2) o;
    -        return Objects.equals(v1, tuple.v1) && Objects.equals(v2, tuple.v2);
    -    }
    -
    -    @Override
    -    public int hashCode() {
    -        return Objects.hash(v1, v2);
    -    }
    -
    -    @Override
    -    public String toString() {
    -        return "Tuple2{v1=" + v1 + ", v2=" + v2 + '}';
    -    }
    -}
    diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumer.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumer.java
    index f84fb88c23..bb5ab1443e 100644
    --- a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumer.java
    +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumer.java
    @@ -22,8 +22,6 @@
     import java.util.Iterator;
     import java.util.Map;
     import java.util.Optional;
    -import java.util.OptionalInt;
    -import java.util.OptionalLong;
     import java.util.concurrent.CountDownLatch;
     import java.util.regex.Pattern;
     import java.util.Collections;
    @@ -68,11 +66,8 @@ public static void main(String[] args) throws Exception {
     
         public static void run(ConsoleConsumerOptions opts) {
             messageCount = 0;
    -        long timeoutMs = opts.timeoutMs() >= 0 ? opts.timeoutMs() : Long.MAX_VALUE;
             Consumer consumer = new KafkaConsumer<>(opts.consumerProps(), new ByteArrayDeserializer(), new ByteArrayDeserializer());
    -        ConsumerWrapper consumerWrapper = opts.partitionArg().isPresent()
    -            ? new ConsumerWrapper(Optional.of(opts.topicArg()), opts.partitionArg(), OptionalLong.of(opts.offsetArg()), Optional.empty(), consumer, timeoutMs)
    -            : new ConsumerWrapper(Optional.of(opts.topicArg()), OptionalInt.empty(), OptionalLong.empty(), Optional.ofNullable(opts.includedTopicsArg()), consumer, timeoutMs);
    +        ConsumerWrapper consumerWrapper = new ConsumerWrapper(opts, consumer);
     
             addShutdownHook(consumerWrapper, opts);
     
    @@ -148,43 +143,25 @@ static boolean checkErr(PrintStream output) {
         }
     
         public static class ConsumerWrapper {
    -        final Optional topic;
    -        final OptionalInt partitionId;
    -        final OptionalLong offset;
    -        final Optional includedTopics;
    -        final Consumer consumer;
    -        final long timeoutMs;
             final Time time = Time.SYSTEM;
    +        final long timeoutMs;
    +        final Consumer consumer;
     
             Iterator> recordIter = Collections.emptyIterator();
     
    -        public ConsumerWrapper(Optional topic,
    -                               OptionalInt partitionId,
    -                               OptionalLong offset,
    -                               Optional includedTopics,
    -                               Consumer consumer,
    -                               long timeoutMs) {
    -            this.topic = topic;
    -            this.partitionId = partitionId;
    -            this.offset = offset;
    -            this.includedTopics = includedTopics;
    +        public ConsumerWrapper(ConsoleConsumerOptions opts, Consumer consumer) {
                 this.consumer = consumer;
    -            this.timeoutMs = timeoutMs;
    -
    -            if (topic.isPresent() && partitionId.isPresent() && offset.isPresent() && !includedTopics.isPresent()) {
    -                seek(topic.get(), partitionId.getAsInt(), offset.getAsLong());
    -            } else if (topic.isPresent() && partitionId.isPresent() && !offset.isPresent() && !includedTopics.isPresent()) {
    -                // default to latest if no offset is provided
    -                seek(topic.get(), partitionId.getAsInt(), ListOffsetsRequest.LATEST_TIMESTAMP);
    -            } else if (topic.isPresent() && !partitionId.isPresent() && !offset.isPresent() && !includedTopics.isPresent()) {
    -                consumer.subscribe(Collections.singletonList(topic.get()));
    -            } else if (!topic.isPresent() && !partitionId.isPresent() && !offset.isPresent() && includedTopics.isPresent()) {
    -                consumer.subscribe(Pattern.compile(includedTopics.get()));
    +            timeoutMs = opts.timeoutMs();
    +            Optional topic = opts.topicArg();
    +
    +            if (topic.isPresent()) {
    +                if (opts.partitionArg().isPresent()) {
    +                    seek(topic.get(), opts.partitionArg().getAsInt(), opts.offsetArg());
    +                } else {
    +                    consumer.subscribe(Collections.singletonList(topic.get()));
    +                }
                 } else {
    -                throw new IllegalArgumentException("An invalid combination of arguments is provided. " +
    -                        "Exactly one of 'topic' or 'include' 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.");
    +                opts.includedTopicsArg().ifPresent(topics -> consumer.subscribe(Pattern.compile(topics)));
                 }
             }
     
    diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java
    index a713afb2bf..aa37919515 100644
    --- a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java
    +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java
    @@ -34,7 +34,7 @@
     import java.util.List;
     import java.util.Locale;
     import java.util.Map;
    -import java.util.Objects;
    +import java.util.Optional;
     import java.util.OptionalInt;
     import java.util.Properties;
     import java.util.Random;
    @@ -55,7 +55,7 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
         private final OptionSpec messageFormatterConfigOpt;
         private final OptionSpec resetBeginningOpt;
         private final OptionSpec maxMessagesOpt;
    -    private final OptionSpec timeoutMsOpt;
    +    private final OptionSpec timeoutMsOpt;
         private final OptionSpec skipMessageOnErrorOpt;
         private final OptionSpec bootstrapServerOpt;
         private final OptionSpec keyDeserializerOpt;
    @@ -66,6 +66,7 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
     
         private final Properties consumerProps;
         private final long offset;
    +    private final long timeoutMs;
         private final MessageFormatter formatter;
     
         public ConsoleConsumerOptions(String[] args) throws IOException {
    @@ -139,7 +140,7 @@ public ConsoleConsumerOptions(String[] args) throws IOException {
             timeoutMsOpt = parser.accepts("timeout-ms", "If specified, exit if no message is available for consumption for the specified interval.")
                     .withRequiredArg()
                     .describedAs("timeout_ms")
    -                .ofType(Integer.class);
    +                .ofType(Long.class);
             skipMessageOnErrorOpt = parser.accepts("skip-message-on-error", "If there is an error when processing a message, " +
                     "skip it instead of halt.");
             bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The server(s) to connect to.")
    @@ -184,12 +185,13 @@ public ConsoleConsumerOptions(String[] args) throws IOException {
             Set groupIdsProvided = checkConsumerGroup(consumerPropsFromFile, extraConsumerProps);
             consumerProps = buildConsumerProps(consumerPropsFromFile, extraConsumerProps, groupIdsProvided);
             offset = parseOffset();
    +        timeoutMs = parseTimeoutMs();
             formatter = buildFormatter();
         }
     
         private void checkRequiredArgs() {
    -        List topicOrFilterArgs = new ArrayList<>(Arrays.asList(topicArg(), includedTopicsArg()));
    -        topicOrFilterArgs.removeIf(Objects::isNull);
    +        List> topicOrFilterArgs = new ArrayList<>(Arrays.asList(topicArg(), includedTopicsArg()));
    +        topicOrFilterArgs.removeIf(arg -> !arg.isPresent());
             // user need to specify value for either --topic or one of the include filters options (--include or --whitelist)
             if (topicOrFilterArgs.size() != 1) {
                 CommandLineUtils.printUsageAndExit(parser, "Exactly one of --include/--topic is required. " +
    @@ -322,6 +324,11 @@ private void invalidOffset(String offset) {
                     "'earliest', 'latest', or a non-negative long.");
         }
     
    +    private long parseTimeoutMs() {
    +        long timeout = options.has(timeoutMsOpt) ? options.valueOf(timeoutMsOpt) : -1;
    +        return timeout >= 0 ? timeout : Long.MAX_VALUE;
    +    }
    +
         private MessageFormatter buildFormatter() {
             MessageFormatter formatter = null;
             try {
    @@ -365,16 +372,16 @@ OptionalInt partitionArg() {
             return OptionalInt.empty();
         }
     
    -    String topicArg() {
    -        return options.valueOf(topicOpt);
    +    Optional topicArg() {
    +        return options.has(topicOpt) ? Optional.of(options.valueOf(topicOpt)) : Optional.empty();
         }
     
         int maxMessages() {
             return options.has(maxMessagesOpt) ? options.valueOf(maxMessagesOpt) : -1;
         }
     
    -    int timeoutMs() {
    -        return options.has(timeoutMsOpt) ? options.valueOf(timeoutMsOpt) : -1;
    +    long timeoutMs() {
    +        return timeoutMs;
         }
     
         boolean enableSystestEventsLogging() {
    @@ -385,10 +392,10 @@ String bootstrapServer() {
             return options.valueOf(bootstrapServerOpt);
         }
     
    -    String includedTopicsArg() {
    +    Optional includedTopicsArg() {
             return options.has(includeOpt)
    -                ? options.valueOf(includeOpt)
    -                : options.valueOf(whitelistOpt);
    +                ? Optional.of(options.valueOf(includeOpt))
    +                : Optional.ofNullable(options.valueOf(whitelistOpt));
         }
     
         Properties formatterArgs() throws IOException {
    diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java
    new file mode 100644
    index 0000000000..5aafde1c0f
    --- /dev/null
    +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java
    @@ -0,0 +1,1234 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements. See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License. You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.kafka.tools.consumer.group;
    +
    +import com.fasterxml.jackson.core.JsonProcessingException;
    +import com.fasterxml.jackson.databind.ObjectReader;
    +import com.fasterxml.jackson.databind.ObjectWriter;
    +import joptsimple.OptionException;
    +import org.apache.kafka.clients.CommonClientConfigs;
    +import org.apache.kafka.clients.admin.AbstractOptions;
    +import org.apache.kafka.clients.admin.Admin;
    +import org.apache.kafka.clients.admin.AlterConsumerGroupOffsetsOptions;
    +import org.apache.kafka.clients.admin.ConsumerGroupDescription;
    +import org.apache.kafka.clients.admin.ConsumerGroupListing;
    +import org.apache.kafka.clients.admin.DeleteConsumerGroupOffsetsOptions;
    +import org.apache.kafka.clients.admin.DeleteConsumerGroupOffsetsResult;
    +import org.apache.kafka.clients.admin.DeleteConsumerGroupsOptions;
    +import org.apache.kafka.clients.admin.DescribeConsumerGroupsOptions;
    +import org.apache.kafka.clients.admin.DescribeTopicsOptions;
    +import org.apache.kafka.clients.admin.DescribeTopicsResult;
    +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions;
    +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsSpec;
    +import org.apache.kafka.clients.admin.ListConsumerGroupsOptions;
    +import org.apache.kafka.clients.admin.ListConsumerGroupsResult;
    +import org.apache.kafka.clients.admin.ListOffsetsOptions;
    +import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
    +import org.apache.kafka.clients.admin.MemberDescription;
    +import org.apache.kafka.clients.admin.OffsetSpec;
    +import org.apache.kafka.clients.admin.TopicDescription;
    +import org.apache.kafka.clients.consumer.OffsetAndMetadata;
    +import org.apache.kafka.common.ConsumerGroupState;
    +import org.apache.kafka.common.GroupType;
    +import org.apache.kafka.common.KafkaException;
    +import org.apache.kafka.common.KafkaFuture;
    +import org.apache.kafka.common.Node;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.protocol.Errors;
    +import org.apache.kafka.common.requests.ListOffsetsResponse;
    +import org.apache.kafka.common.utils.Utils;
    +import org.apache.kafka.server.util.CommandLineUtils;
    +import org.apache.kafka.tools.ToolsUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.text.ParseException;
    +import java.time.Duration;
    +import java.time.Instant;
    +import java.util.AbstractMap.SimpleImmutableEntry;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Optional;
    +import java.util.OptionalInt;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.concurrent.ExecutionException;
    +import java.util.function.BiFunction;
    +import java.util.function.Function;
    +import java.util.function.ToIntFunction;
    +import java.util.stream.Collectors;
    +import java.util.stream.Stream;
    +
    +public class ConsumerGroupCommand {
    +    private static final Logger LOGGER = LoggerFactory.getLogger(ConsumerGroupCommand.class);
    +
    +    static final String MISSING_COLUMN_VALUE = "-";
    +
    +    public static void main(String[] args) {
    +        ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(args);
    +        try {
    +            // should have exactly one action
    +            long actions = Stream.of(opts.listOpt, opts.describeOpt, opts.deleteOpt, opts.resetOffsetsOpt, opts.deleteOffsetsOpt).filter(opts.options::has).count();
    +            if (actions != 1)
    +                CommandLineUtils.printUsageAndExit(opts.parser, "Command must include exactly one action: --list, --describe, --delete, --reset-offsets, --delete-offsets");
    +
    +            run(opts);
    +        } catch (OptionException e) {
    +            CommandLineUtils.printUsageAndExit(opts.parser, e.getMessage());
    +        }
    +    }
    +
    +    static void run(ConsumerGroupCommandOptions opts) {
    +        try (ConsumerGroupService consumerGroupService = new ConsumerGroupService(opts, Collections.emptyMap())) {
    +            if (opts.options.has(opts.listOpt))
    +                consumerGroupService.listGroups();
    +            else if (opts.options.has(opts.describeOpt))
    +                consumerGroupService.describeGroups();
    +            else if (opts.options.has(opts.deleteOpt))
    +                consumerGroupService.deleteGroups();
    +            else if (opts.options.has(opts.resetOffsetsOpt)) {
    +                Map> offsetsToReset = consumerGroupService.resetOffsets();
    +                if (opts.options.has(opts.exportOpt)) {
    +                    String exported = consumerGroupService.exportOffsetsToCsv(offsetsToReset);
    +                    System.out.println(exported);
    +                } else
    +                    printOffsetsToReset(offsetsToReset);
    +            } else if (opts.options.has(opts.deleteOffsetsOpt)) {
    +                consumerGroupService.deleteOffsets();
    +            }
    +        } catch (IllegalArgumentException e) {
    +            CommandLineUtils.printUsageAndExit(opts.parser, e.getMessage());
    +        } catch (Throwable e) {
    +            printError("Executing consumer group command failed due to " + e.getMessage(), Optional.of(e));
    +        }
    +    }
    +
    +    static Set consumerGroupStatesFromString(String input) {
    +        Set parsedStates = Arrays.stream(input.split(",")).map(s -> ConsumerGroupState.parse(s.trim())).collect(Collectors.toSet());
    +        if (parsedStates.contains(ConsumerGroupState.UNKNOWN)) {
    +            Collection validStates = Arrays.stream(ConsumerGroupState.values()).filter(s -> s != ConsumerGroupState.UNKNOWN).collect(Collectors.toList());
    +            throw new IllegalArgumentException("Invalid state list '" + input + "'. Valid states are: " + Utils.join(validStates, ", "));
    +        }
    +        return parsedStates;
    +    }
    +
    +    @SuppressWarnings("Regexp")
    +    static Set consumerGroupTypesFromString(String input) {
    +        Set parsedTypes = Stream.of(input.toLowerCase().split(",")).map(s -> GroupType.parse(s.trim())).collect(Collectors.toSet());
    +        if (parsedTypes.contains(GroupType.UNKNOWN)) {
    +            List validTypes = Arrays.stream(GroupType.values()).filter(t -> t != GroupType.UNKNOWN).map(Object::toString).collect(Collectors.toList());
    +            throw new IllegalArgumentException("Invalid types list '" + input + "'. Valid types are: " + String.join(", ", validTypes));
    +        }
    +        return parsedTypes;
    +    }
    +
    +    static void printError(String msg, Optional e) {
    +        System.out.println("\nError: " + msg);
    +        e.ifPresent(Throwable::printStackTrace);
    +    }
    +
    +    static void printOffsetsToReset(Map> groupAssignmentsToReset) {
    +        String format = "%-30s %-30s %-10s %-15s";
    +        if (!groupAssignmentsToReset.isEmpty())
    +            System.out.printf("\n" + format, "GROUP", "TOPIC", "PARTITION", "NEW-OFFSET");
    +
    +        groupAssignmentsToReset.forEach((groupId, assignment) ->
    +            assignment.forEach((consumerAssignment, offsetAndMetadata) ->
    +                System.out.printf(format,
    +                    groupId,
    +                    consumerAssignment.topic(),
    +                    consumerAssignment.partition(),
    +                    offsetAndMetadata.offset())));
    +    }
    +
    +    @SuppressWarnings("ClassFanOutComplexity")
    +    static class ConsumerGroupService implements AutoCloseable {
    +        final ConsumerGroupCommandOptions opts;
    +        final Map configOverrides;
    +        private final Admin adminClient;
    +
    +        ConsumerGroupService(ConsumerGroupCommandOptions opts, Map configOverrides) {
    +            this.opts = opts;
    +            this.configOverrides = configOverrides;
    +            try {
    +                this.adminClient = createAdminClient(configOverrides);
    +            } catch (IOException e) {
    +                throw new RuntimeException(e);
    +            }
    +        }
    +
    +        Optional>> resetPlanFromFile() {
    +            if (opts.options.has(opts.resetFromFileOpt)) {
    +                try {
    +                    String resetPlanPath = opts.options.valueOf(opts.resetFromFileOpt);
    +                    String resetPlanCsv = Utils.readFileAsString(resetPlanPath);
    +                    Map> resetPlan = parseResetPlan(resetPlanCsv);
    +                    return Optional.of(resetPlan);
    +                } catch (IOException e) {
    +                    throw new RuntimeException(e);
    +                }
    +            } else return Optional.empty();
    +        }
    +
    +        void listGroups() throws ExecutionException, InterruptedException {
    +            boolean includeType = opts.options.has(opts.typeOpt);
    +            boolean includeState = opts.options.has(opts.stateOpt);
    +
    +            if (includeType || includeState) {
    +                Set types = typeValues();
    +                Set states = stateValues();
    +                List listings = listConsumerGroupsWithFilters(types, states);
    +
    +                printGroupInfo(listings, includeType, includeState);
    +            } else {
    +                listConsumerGroups().forEach(System.out::println);
    +            }
    +        }
    +
    +        private Set stateValues() {
    +            String stateValue = opts.options.valueOf(opts.stateOpt);
    +            return (stateValue == null || stateValue.isEmpty())
    +                ? Collections.emptySet()
    +                : consumerGroupStatesFromString(stateValue);
    +        }
    +
    +        private Set typeValues() {
    +            String typeValue = opts.options.valueOf(opts.typeOpt);
    +            return (typeValue == null || typeValue.isEmpty())
    +                ? Collections.emptySet()
    +                : consumerGroupTypesFromString(typeValue);
    +        }
    +
    +        private void printGroupInfo(List groups, boolean includeType, boolean includeState) {
    +            Function groupId = ConsumerGroupListing::groupId;
    +            Function groupType = groupListing -> groupListing.type().orElse(GroupType.UNKNOWN).toString();
    +            Function groupState = groupListing -> groupListing.state().orElse(ConsumerGroupState.UNKNOWN).toString();
    +
    +            OptionalInt maybeMax = groups.stream().mapToInt(groupListing -> Math.max(15, groupId.apply(groupListing).length())).max();
    +            int maxGroupLen = maybeMax.orElse(15) + 10;
    +            String format = "%-" + maxGroupLen + "s";
    +            List header = new ArrayList<>();
    +            header.add("GROUP");
    +            List> extractors = new ArrayList<>();
    +            extractors.add(groupId);
    +
    +            if (includeType) {
    +                header.add("TYPE");
    +                extractors.add(groupType);
    +                format += " %-20s";
    +            }
    +
    +            if (includeState) {
    +                header.add("STATE");
    +                extractors.add(groupState);
    +                format += " %-20s";
    +            }
    +
    +            System.out.printf(format + "%n", header.toArray(new Object[0]));
    +
    +            for (ConsumerGroupListing groupListing : groups) {
    +                Object[] info = extractors.stream().map(extractor -> extractor.apply(groupListing)).toArray(Object[]::new);
    +                System.out.printf(format + "%n", info);
    +            }
    +        }
    +
    +        List listConsumerGroups() {
    +            try {
    +                ListConsumerGroupsResult result = adminClient.listConsumerGroups(withTimeoutMs(new ListConsumerGroupsOptions()));
    +                Collection listings = result.all().get();
    +                return listings.stream().map(ConsumerGroupListing::groupId).collect(Collectors.toList());
    +            } catch (InterruptedException | ExecutionException e) {
    +                throw new RuntimeException(e);
    +            }
    +        }
    +
    +        List listConsumerGroupsWithFilters(Set types, Set states) throws ExecutionException, InterruptedException {
    +            ListConsumerGroupsOptions listConsumerGroupsOptions = withTimeoutMs(new ListConsumerGroupsOptions());
    +            listConsumerGroupsOptions
    +                .inStates(states)
    +                .withTypes(types);
    +            ListConsumerGroupsResult result = adminClient.listConsumerGroups(listConsumerGroupsOptions);
    +            return new ArrayList<>(result.all().get());
    +        }
    +
    +        private boolean shouldPrintMemberState(String group, Optional state, Optional numRows) {
    +            // 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.
    +            if (!numRows.isPresent()) {
    +                printError("The consumer group '" + group + "' does not exist.", Optional.empty());
    +                return false;
    +            }
    +
    +            int num = numRows.get();
    +
    +            String state0 = state.orElse("NONE");
    +            switch (state0) {
    +                case "Dead":
    +                    printError("Consumer group '" + group + "' does not exist.", Optional.empty());
    +                    break;
    +                case "Empty":
    +                    System.err.println("\nConsumer group '" + group + "' has no active members.");
    +                    break;
    +                case "PreparingRebalance":
    +                case "CompletingRebalance":
    +                case "Assigning":
    +                case "Reconciling":
    +                    System.err.println("\nWarning: Consumer group '" + group + "' is rebalancing.");
    +                    break;
    +                case "Stable":
    +                    break;
    +                default:
    +                    // the control should never reach here
    +                    throw new KafkaException("Expected a valid consumer group state, but found '" + state0 + "'.");
    +            }
    +
    +            return !state0.contains("Dead") && num > 0;
    +        }
    +
    +        private Optional size(Optional> colOpt) {
    +            return colOpt.map(Collection::size);
    +        }
    +
    +        private void printOffsets(Map, Optional>>> offsets) {
    +            offsets.forEach((groupId, tuple) -> {
    +                Optional state = tuple.getKey();
    +                Optional> assignments = tuple.getValue();
    +
    +                if (shouldPrintMemberState(groupId, state, size(assignments))) {
    +                    String format = printOffsetFormat(assignments);
    +
    +                    System.out.printf(format, "GROUP", "TOPIC", "PARTITION", "CURRENT-OFFSET", "LOG-END-OFFSET", "LAG", "CONSUMER-ID", "HOST", "CLIENT-ID");
    +
    +                    if (assignments.isPresent()) {
    +                        Collection consumerAssignments = assignments.get();
    +                        for (PartitionAssignmentState consumerAssignment : consumerAssignments) {
    +                            System.out.printf(format,
    +                                consumerAssignment.group,
    +                                consumerAssignment.topic.orElse(MISSING_COLUMN_VALUE), consumerAssignment.partition.map(Object::toString).orElse(MISSING_COLUMN_VALUE),
    +                                consumerAssignment.offset.map(Object::toString).orElse(MISSING_COLUMN_VALUE), consumerAssignment.logEndOffset.map(Object::toString).orElse(MISSING_COLUMN_VALUE),
    +                                consumerAssignment.lag.map(Object::toString).orElse(MISSING_COLUMN_VALUE), consumerAssignment.consumerId.orElse(MISSING_COLUMN_VALUE),
    +                                consumerAssignment.host.orElse(MISSING_COLUMN_VALUE), consumerAssignment.clientId.orElse(MISSING_COLUMN_VALUE)
    +                            );
    +                        }
    +                    }
    +                }
    +            });
    +        }
    +
    +        private static String printOffsetFormat(Optional> assignments) {
    +            // find proper columns width
    +            int maxGroupLen = 15, maxTopicLen = 15, maxConsumerIdLen = 15, maxHostLen = 15;
    +            if (assignments.isPresent()) {
    +                Collection consumerAssignments = assignments.get();
    +                for (PartitionAssignmentState consumerAssignment : consumerAssignments) {
    +                    maxGroupLen = Math.max(maxGroupLen, consumerAssignment.group.length());
    +                    maxTopicLen = Math.max(maxTopicLen, consumerAssignment.topic.orElse(MISSING_COLUMN_VALUE).length());
    +                    maxConsumerIdLen = Math.max(maxConsumerIdLen, consumerAssignment.consumerId.orElse(MISSING_COLUMN_VALUE).length());
    +                    maxHostLen = Math.max(maxHostLen, consumerAssignment.host.orElse(MISSING_COLUMN_VALUE).length());
    +
    +                }
    +            }
    +
    +            String format = "\n%" + (-maxGroupLen) + "s %" + (-maxTopicLen) + "s %-10s %-15s %-15s %-15s %" + (-maxConsumerIdLen) + "s %" + (-maxHostLen) + "s %s";
    +            return format;
    +        }
    +
    +        private void printMembers(Map, Optional>>> members, boolean verbose) {
    +            members.forEach((groupId, tuple) -> {
    +                Optional state = tuple.getKey();
    +                Optional> assignments = tuple.getValue();
    +                int maxGroupLen = 15, maxConsumerIdLen = 15, maxGroupInstanceIdLen = 17, maxHostLen = 15, maxClientIdLen = 15;
    +                boolean includeGroupInstanceId = false;
    +
    +                if (shouldPrintMemberState(groupId, state, size(assignments))) {
    +                    // find proper columns width
    +                    if (assignments.isPresent()) {
    +                        for (MemberAssignmentState memberAssignment : assignments.get()) {
    +                            maxGroupLen = Math.max(maxGroupLen, memberAssignment.group.length());
    +                            maxConsumerIdLen = Math.max(maxConsumerIdLen, memberAssignment.consumerId.length());
    +                            maxGroupInstanceIdLen =  Math.max(maxGroupInstanceIdLen, memberAssignment.groupInstanceId.length());
    +                            maxHostLen = Math.max(maxHostLen, memberAssignment.host.length());
    +                            maxClientIdLen = Math.max(maxClientIdLen, memberAssignment.clientId.length());
    +                            includeGroupInstanceId = includeGroupInstanceId || !memberAssignment.groupInstanceId.isEmpty();
    +                        }
    +                    }
    +                }
    +
    +                String format0 = "%" + -maxGroupLen + "s %" + -maxConsumerIdLen + "s %" + -maxGroupInstanceIdLen + "s %" + -maxHostLen + "s %" + -maxClientIdLen + "s %-15s ";
    +                String format1 = "%" + -maxGroupLen + "s %" + -maxConsumerIdLen + "s %" + -maxHostLen + "s %" + -maxClientIdLen + "s %-15s ";
    +
    +                if (includeGroupInstanceId) {
    +                    System.out.printf("\n" + format0, "GROUP", "CONSUMER-ID", "GROUP-INSTANCE-ID", "HOST", "CLIENT-ID", "#PARTITIONS");
    +                } else {
    +                    System.out.printf("\n" + format1, "GROUP", "CONSUMER-ID", "HOST", "CLIENT-ID", "#PARTITIONS");
    +                }
    +                if (verbose)
    +                    System.out.printf("%s", "ASSIGNMENT");
    +                System.out.println();
    +
    +                if (assignments.isPresent()) {
    +                    for (MemberAssignmentState memberAssignment : assignments.get()) {
    +                        if (includeGroupInstanceId) {
    +                            System.out.printf(format0, memberAssignment.group, memberAssignment.consumerId,
    +                                memberAssignment.groupInstanceId, memberAssignment.host, memberAssignment.clientId,
    +                                memberAssignment.numPartitions);
    +                        } else {
    +                            System.out.printf(format1, memberAssignment.group, memberAssignment.consumerId,
    +                                memberAssignment.host, memberAssignment.clientId, memberAssignment.numPartitions);
    +                        }
    +                        if (verbose) {
    +                            String partitions;
    +
    +                            if (memberAssignment.assignment.isEmpty())
    +                                partitions = MISSING_COLUMN_VALUE;
    +                            else {
    +                                Map> grouped = new HashMap<>();
    +                                memberAssignment.assignment.forEach(
    +                                    tp -> grouped.computeIfAbsent(tp.topic(), key -> new ArrayList<>()).add(tp));
    +                                partitions = grouped.values().stream().map(topicPartitions ->
    +                                    topicPartitions.stream().map(TopicPartition::partition).map(Object::toString).sorted().collect(Collectors.joining(",", "(", ")"))
    +                                ).sorted().collect(Collectors.joining(", "));
    +                            }
    +                            System.out.printf("%s", partitions);
    +                        }
    +                        System.out.println();
    +                    }
    +                }
    +            });
    +        }
    +
    +        private void printStates(Map states) {
    +            states.forEach((groupId, state) -> {
    +                if (shouldPrintMemberState(groupId, Optional.of(state.state), Optional.of(1))) {
    +                    String coordinator = state.coordinator.host() + ":" + state.coordinator.port() + "  (" + state.coordinator.idString() + ")";
    +                    int coordinatorColLen = Math.max(25, coordinator.length());
    +
    +                    String format = "\n%" + -coordinatorColLen + "s %-25s %-20s %-15s %s";
    +
    +                    System.out.printf(format, "GROUP", "COORDINATOR (ID)", "ASSIGNMENT-STRATEGY", "STATE", "#MEMBERS");
    +                    System.out.printf(format, state.group, coordinator, state.assignmentStrategy, state.state, state.numMembers);
    +                    System.out.println();
    +                }
    +            });
    +        }
    +
    +        void describeGroups() throws Exception {
    +            Collection groupIds = opts.options.has(opts.allGroupsOpt)
    +                ? listConsumerGroups()
    +                : opts.options.valuesOf(opts.groupOpt);
    +            boolean membersOptPresent = opts.options.has(opts.membersOpt);
    +            boolean stateOptPresent = opts.options.has(opts.stateOpt);
    +            boolean offsetsOptPresent = opts.options.has(opts.offsetsOpt);
    +            long subActions = Stream.of(membersOptPresent, offsetsOptPresent, stateOptPresent).filter(x -> x).count();
    +
    +            if (subActions == 0 || offsetsOptPresent) {
    +                TreeMap, Optional>>> offsets
    +                    = collectGroupsOffsets(groupIds);
    +                printOffsets(offsets);
    +            } else if (membersOptPresent) {
    +                TreeMap, Optional>>> members
    +                    = collectGroupsMembers(groupIds, opts.options.has(opts.verboseOpt));
    +                printMembers(members, opts.options.has(opts.verboseOpt));
    +            } else {
    +                TreeMap states = collectGroupsState(groupIds);
    +                printStates(states);
    +            }
    +        }
    +
    +        private Collection collectConsumerAssignment(
    +            String group,
    +            Optional coordinator,
    +            Collection topicPartitions,
    +            Function> getPartitionOffset,
    +            Optional consumerIdOpt,
    +            Optional hostOpt,
    +            Optional clientIdOpt
    +        ) {
    +            if (topicPartitions.isEmpty()) {
    +                return Collections.singleton(
    +                    new PartitionAssignmentState(group, coordinator, Optional.empty(), Optional.empty(), Optional.empty(),
    +                        getLag(Optional.empty(), Optional.empty()), consumerIdOpt, hostOpt, clientIdOpt, Optional.empty())
    +                );
    +            } else {
    +                List topicPartitionsSorted = topicPartitions.stream().sorted(Comparator.comparingInt(TopicPartition::partition)).collect(Collectors.toList());
    +                return describePartitions(group, coordinator, topicPartitionsSorted, getPartitionOffset, consumerIdOpt, hostOpt, clientIdOpt);
    +            }
    +        }
    +
    +        private Optional getLag(Optional offset, Optional logEndOffset) {
    +            return offset.filter(o -> o != -1).flatMap(offset0 -> logEndOffset.map(end -> end - offset0));
    +        }
    +
    +        private Collection describePartitions(String group,
    +                                                              Optional coordinator,
    +                                                              List topicPartitions,
    +                                                              Function> getPartitionOffset,
    +                                                              Optional consumerIdOpt,
    +                                                              Optional hostOpt,
    +                                                              Optional clientIdOpt) {
    +            BiFunction, PartitionAssignmentState> getDescribePartitionResult = (topicPartition, logEndOffsetOpt) -> {
    +                Optional offset = getPartitionOffset.apply(topicPartition);
    +                return new PartitionAssignmentState(group, coordinator, Optional.of(topicPartition.topic()),
    +                    Optional.of(topicPartition.partition()), offset, getLag(offset, logEndOffsetOpt),
    +                    consumerIdOpt, hostOpt, clientIdOpt, logEndOffsetOpt);
    +            };
    +
    +            return getLogEndOffsets(topicPartitions).entrySet().stream().map(logEndOffsetResult -> {
    +                if (logEndOffsetResult.getValue() instanceof LogOffset)
    +                    return getDescribePartitionResult.apply(
    +                        logEndOffsetResult.getKey(),
    +                        Optional.of(((LogOffset) logEndOffsetResult.getValue()).value)
    +                    );
    +                else if (logEndOffsetResult.getValue() instanceof Unknown)
    +                    return getDescribePartitionResult.apply(logEndOffsetResult.getKey(), Optional.empty());
    +                else if (logEndOffsetResult.getValue() instanceof Ignore)
    +                    return null;
    +
    +                throw new IllegalStateException("Unknown LogOffset subclass: " + logEndOffsetResult.getValue());
    +            }).collect(Collectors.toList());
    +        }
    +
    +        Map> resetOffsets() {
    +            List groupIds = opts.options.has(opts.allGroupsOpt)
    +                ? listConsumerGroups()
    +                : opts.options.valuesOf(opts.groupOpt);
    +
    +            Map> consumerGroups = adminClient.describeConsumerGroups(
    +                groupIds,
    +                withTimeoutMs(new DescribeConsumerGroupsOptions())
    +            ).describedGroups();
    +
    +            Map> result = new HashMap<>();
    +
    +            consumerGroups.forEach((groupId, groupDescription) -> {
    +                try {
    +                    String state = groupDescription.get().state().toString();
    +                    switch (state) {
    +                        case "Empty":
    +                        case "Dead":
    +                            Collection partitionsToReset = getPartitionsToReset(groupId);
    +                            Map preparedOffsets = prepareOffsetsToReset(groupId, partitionsToReset);
    +
    +                            // Dry-run is the default behavior if --execute is not specified
    +                            boolean dryRun = opts.options.has(opts.dryRunOpt) || !opts.options.has(opts.executeOpt);
    +                            if (!dryRun) {
    +                                adminClient.alterConsumerGroupOffsets(
    +                                    groupId,
    +                                    preparedOffsets,
    +                                    withTimeoutMs(new AlterConsumerGroupOffsetsOptions())
    +                                ).all().get();
    +                            }
    +
    +                            result.put(groupId, preparedOffsets);
    +
    +                            break;
    +                        default:
    +                            printError("Assignments can only be reset if the group '" + groupId + "' is inactive, but the current state is " + state + ".", Optional.empty());
    +                            result.put(groupId, Collections.emptyMap());
    +                    }
    +                } catch (InterruptedException | ExecutionException e) {
    +                    throw new RuntimeException(e);
    +                }
    +            });
    +
    +            return result;
    +        }
    +
    +        Entry> deleteOffsets(String groupId, List topics) {
    +            Map partitionLevelResult = new HashMap<>();
    +            Set topicWithPartitions = new HashSet<>();
    +            Set topicWithoutPartitions = new HashSet<>();
    +
    +            for (String topic : topics) {
    +                if (topic.contains(":"))
    +                    topicWithPartitions.add(topic);
    +                else
    +                    topicWithoutPartitions.add(topic);
    +            }
    +
    +            List knownPartitions = topicWithPartitions.stream().flatMap(this::parseTopicsWithPartitions).collect(Collectors.toList());
    +
    +            // Get the partitions of topics that the user did not explicitly specify the partitions
    +            DescribeTopicsResult describeTopicsResult = adminClient.describeTopics(
    +                topicWithoutPartitions,
    +                withTimeoutMs(new DescribeTopicsOptions()));
    +
    +            Iterator unknownPartitions = describeTopicsResult.topicNameValues().entrySet().stream().flatMap(e -> {
    +                String topic = e.getKey();
    +                try {
    +                    return e.getValue().get().partitions().stream().map(partition ->
    +                        new TopicPartition(topic, partition.partition()));
    +                } catch (ExecutionException | InterruptedException err) {
    +                    partitionLevelResult.put(new TopicPartition(topic, -1), err);
    +                    return Stream.empty();
    +                }
    +            }).iterator();
    +
    +            Set partitions = new HashSet<>(knownPartitions);
    +
    +            unknownPartitions.forEachRemaining(partitions::add);
    +
    +            DeleteConsumerGroupOffsetsResult deleteResult = adminClient.deleteConsumerGroupOffsets(
    +                groupId,
    +                partitions,
    +                withTimeoutMs(new DeleteConsumerGroupOffsetsOptions())
    +            );
    +
    +            Errors topLevelException = Errors.NONE;
    +
    +            try {
    +                deleteResult.all().get();
    +            } catch (ExecutionException | InterruptedException e) {
    +                topLevelException = Errors.forException(e.getCause());
    +            }
    +
    +            partitions.forEach(partition -> {
    +                try {
    +                    deleteResult.partitionResult(partition).get();
    +                    partitionLevelResult.put(partition, null);
    +                } catch (ExecutionException | InterruptedException e) {
    +                    partitionLevelResult.put(partition, e);
    +                }
    +            });
    +
    +            return new SimpleImmutableEntry<>(topLevelException, partitionLevelResult);
    +        }
    +
    +        void deleteOffsets() {
    +            String groupId = opts.options.valueOf(opts.groupOpt);
    +            List topics = opts.options.valuesOf(opts.topicOpt);
    +
    +            Entry> res = deleteOffsets(groupId, topics);
    +
    +            Errors topLevelResult = res.getKey();
    +            Map partitionLevelResult = res.getValue();
    +
    +            switch (topLevelResult) {
    +                case NONE:
    +                    System.out.println("Request succeed for deleting offsets with topic " + Utils.mkString(topics.stream(), "", "", ", ") + " group " + groupId);
    +                    break;
    +                case INVALID_GROUP_ID:
    +                    printError("'" + groupId + "' is not valid.", Optional.empty());
    +                    break;
    +                case GROUP_ID_NOT_FOUND:
    +                    printError("'" + groupId + "' does not exist.", Optional.empty());
    +                    break;
    +                case GROUP_AUTHORIZATION_FAILED:
    +                    printError("Access to '" + groupId + "' is not authorized.", Optional.empty());
    +                    break;
    +                case NON_EMPTY_GROUP:
    +                    printError("Deleting offsets of a consumer group '" + groupId + "' is forbidden if the group is not empty.", Optional.empty());
    +                    break;
    +                case GROUP_SUBSCRIBED_TO_TOPIC:
    +                case TOPIC_AUTHORIZATION_FAILED:
    +                case UNKNOWN_TOPIC_OR_PARTITION:
    +                    printError("Encounter some partition level error, see the follow-up details:", Optional.empty());
    +                    break;
    +                default:
    +                    printError("Encounter some unknown error: " + topLevelResult, Optional.empty());
    +            }
    +
    +            String format = "%-30s %-15s %-15s";
    +
    +            System.out.printf("\n" + format, "TOPIC", "PARTITION", "STATUS");
    +            partitionLevelResult.entrySet().stream()
    +                .sorted(Comparator.comparing(e -> e.getKey().topic() + e.getKey().partition()))
    +                .forEach(e -> {
    +                    TopicPartition tp = e.getKey();
    +                    Throwable error = e.getValue();
    +                    System.out.printf(format,
    +                        tp.topic(),
    +                        tp.partition() >= 0 ? tp.partition() : "Not Provided",
    +                        error != null ? "Error: :" + error.getMessage() : "Successful"
    +                    );
    +                });
    +        }
    +
    +        Map describeConsumerGroups(Collection groupIds) throws Exception {
    +            Map res = new HashMap<>();
    +            Map> stringKafkaFutureMap = adminClient.describeConsumerGroups(
    +                groupIds,
    +                withTimeoutMs(new DescribeConsumerGroupsOptions())
    +            ).describedGroups();
    +
    +            for (Entry> e : stringKafkaFutureMap.entrySet()) {
    +                res.put(e.getKey(), e.getValue().get());
    +            }
    +            return res;
    +        }
    +
    +        /**
    +         * Returns the state of the specified consumer group and partition assignment states
    +         */
    +        Entry, Optional>> collectGroupOffsets(String groupId) throws Exception {
    +            return collectGroupsOffsets(Collections.singletonList(groupId)).getOrDefault(groupId, new SimpleImmutableEntry<>(Optional.empty(), Optional.empty()));
    +        }
    +
    +        /**
    +         * Returns states of the specified consumer groups and partition assignment states
    +         */
    +        TreeMap, Optional>>> collectGroupsOffsets(Collection groupIds) throws Exception {
    +            Map consumerGroups = describeConsumerGroups(groupIds);
    +            TreeMap, Optional>>> groupOffsets = new TreeMap<>();
    +
    +            consumerGroups.forEach((groupId, consumerGroup) -> {
    +                ConsumerGroupState state = consumerGroup.state();
    +                Map committedOffsets = getCommittedOffsets(groupId);
    +                // The admin client returns `null` as a value to indicate that there is not committed offset for a partition.
    +                Function> getPartitionOffset = tp -> Optional.ofNullable(committedOffsets.get(tp)).map(OffsetAndMetadata::offset);
    +                List assignedTopicPartitions = new ArrayList<>();
    +                Comparator comparator =
    +                    Comparator.comparingInt(m -> m.assignment().topicPartitions().size()).reversed();
    +                List rowsWithConsumer = new ArrayList<>();
    +                consumerGroup.members().stream().filter(m -> !m.assignment().topicPartitions().isEmpty())
    +                    .sorted(comparator)
    +                    .forEach(consumerSummary -> {
    +                        Set topicPartitions = consumerSummary.assignment().topicPartitions();
    +                        assignedTopicPartitions.addAll(topicPartitions);
    +                        rowsWithConsumer.addAll(collectConsumerAssignment(
    +                            groupId,
    +                            Optional.of(consumerGroup.coordinator()),
    +                            topicPartitions,
    +                            getPartitionOffset,
    +                            Optional.of(consumerSummary.consumerId()),
    +                            Optional.of(consumerSummary.host()),
    +                            Optional.of(consumerSummary.clientId()))
    +                        );
    +                    });
    +                Map unassignedPartitions = new HashMap<>();
    +                committedOffsets.entrySet().stream().filter(e -> !assignedTopicPartitions.contains(e.getKey()))
    +                    .forEach(e -> unassignedPartitions.put(e.getKey(), e.getValue()));
    +                Collection rowsWithoutConsumer = !unassignedPartitions.isEmpty()
    +                    ? collectConsumerAssignment(
    +                        groupId,
    +                        Optional.of(consumerGroup.coordinator()),
    +                        unassignedPartitions.keySet(),
    +                        getPartitionOffset,
    +                        Optional.of(MISSING_COLUMN_VALUE),
    +                        Optional.of(MISSING_COLUMN_VALUE),
    +                        Optional.of(MISSING_COLUMN_VALUE))
    +                    : Collections.emptyList();
    +
    +                rowsWithConsumer.addAll(rowsWithoutConsumer);
    +
    +                groupOffsets.put(groupId, new SimpleImmutableEntry<>(Optional.of(state.toString()), Optional.of(rowsWithConsumer)));
    +            });
    +
    +            return groupOffsets;
    +        }
    +
    +        Entry, Optional>> collectGroupMembers(String groupId, boolean verbose) throws Exception {
    +            return collectGroupsMembers(Collections.singleton(groupId), verbose).get(groupId);
    +        }
    +
    +        TreeMap, Optional>>> collectGroupsMembers(Collection groupIds, boolean verbose) throws Exception {
    +            Map consumerGroups = describeConsumerGroups(groupIds);
    +            TreeMap, Optional>>> res = new TreeMap<>();
    +
    +            consumerGroups.forEach((groupId, consumerGroup) -> {
    +                String state = consumerGroup.state().toString();
    +                List memberAssignmentStates = consumerGroup.members().stream().map(consumer ->
    +                    new MemberAssignmentState(
    +                        groupId,
    +                        consumer.consumerId(),
    +                        consumer.host(),
    +                        consumer.clientId(),
    +                        consumer.groupInstanceId().orElse(""),
    +                        consumer.assignment().topicPartitions().size(),
    +                        new ArrayList<>(verbose ? consumer.assignment().topicPartitions() : Collections.emptySet())
    +                )).collect(Collectors.toList());
    +                res.put(groupId, new SimpleImmutableEntry<>(Optional.of(state), Optional.of(memberAssignmentStates)));
    +            });
    +            return res;
    +        }
    +
    +        GroupState collectGroupState(String groupId) throws Exception {
    +            return collectGroupsState(Collections.singleton(groupId)).get(groupId);
    +        }
    +
    +        TreeMap collectGroupsState(Collection groupIds) throws Exception {
    +            Map consumerGroups = describeConsumerGroups(groupIds);
    +            TreeMap res = new TreeMap<>();
    +            consumerGroups.forEach((groupId, groupDescription) ->
    +                res.put(groupId, new GroupState(
    +                    groupId,
    +                    groupDescription.coordinator(),
    +                    groupDescription.partitionAssignor(),
    +                    groupDescription.state().toString(),
    +                    groupDescription.members().size()
    +            )));
    +            return res;
    +        }
    +
    +        private Map getLogEndOffsets(Collection topicPartitions) {
    +            return getLogOffsets(topicPartitions, OffsetSpec.latest());
    +        }
    +
    +        private Map getLogStartOffsets(Collection topicPartitions) {
    +            return getLogOffsets(topicPartitions, OffsetSpec.earliest());
    +        }
    +
    +        private Map getLogOffsets(Collection topicPartitions, OffsetSpec offsetSpec) {
    +            try {
    +                Map startOffsets = topicPartitions.stream()
    +                    .collect(Collectors.toMap(Function.identity(), tp -> offsetSpec));
    +
    +                Map offsets = adminClient.listOffsets(
    +                    startOffsets,
    +                    withTimeoutMs(new ListOffsetsOptions())
    +                ).all().get();
    +
    +                return topicPartitions.stream().collect(Collectors.toMap(
    +                    Function.identity(),
    +                    tp -> offsets.containsKey(tp)
    +                        ? new LogOffset(offsets.get(tp).offset())
    +                        : new Unknown()
    +                ));
    +            } catch (InterruptedException | ExecutionException e) {
    +                throw new RuntimeException(e);
    +            }
    +        }
    +
    +        private Map getLogTimestampOffsets(Collection topicPartitions, long timestamp) {
    +            try {
    +                Map timestampOffsets = topicPartitions.stream()
    +                    .collect(Collectors.toMap(Function.identity(), tp -> OffsetSpec.forTimestamp(timestamp)));
    +
    +                Map offsets = adminClient.listOffsets(
    +                    timestampOffsets,
    +                    withTimeoutMs(new ListOffsetsOptions())
    +                ).all().get();
    +
    +                Map successfulOffsetsForTimes = new HashMap<>();
    +                Map unsuccessfulOffsetsForTimes = new HashMap<>();
    +
    +                offsets.forEach((tp, offsetsResultInfo) -> {
    +                    if (offsetsResultInfo.offset() != ListOffsetsResponse.UNKNOWN_OFFSET)
    +                        successfulOffsetsForTimes.put(tp, offsetsResultInfo);
    +                    else
    +                        unsuccessfulOffsetsForTimes.put(tp, offsetsResultInfo);
    +                });
    +
    +                Map successfulLogTimestampOffsets = successfulOffsetsForTimes.entrySet().stream()
    +                    .collect(Collectors.toMap(Entry::getKey, e -> new LogOffset(e.getValue().offset())));
    +
    +                unsuccessfulOffsetsForTimes.forEach((tp, offsetResultInfo) ->
    +                    System.out.println("\nWarn: Partition " + tp.partition() + " from topic " + tp.topic() +
    +                    " is empty. Falling back to latest known offset."));
    +
    +                successfulLogTimestampOffsets.putAll(getLogEndOffsets(unsuccessfulOffsetsForTimes.keySet()));
    +
    +                return successfulLogTimestampOffsets;
    +            } catch (InterruptedException | ExecutionException e) {
    +                throw new RuntimeException(e);
    +            }
    +        }
    +
    +        @Override
    +        public void close() {
    +            adminClient.close();
    +        }
    +
    +        // Visibility for testing
    +        protected Admin createAdminClient(Map configOverrides) throws IOException {
    +            Properties props = opts.options.has(opts.commandConfigOpt) ? Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt)) : new Properties();
    +            props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt));
    +            props.putAll(configOverrides);
    +            return Admin.create(props);
    +        }
    +
    +        private > T withTimeoutMs(T options) {
    +            int t = opts.options.valueOf(opts.timeoutMsOpt).intValue();
    +            return options.timeoutMs(t);
    +        }
    +
    +        private Stream parseTopicsWithPartitions(String topicArg) {
    +            ToIntFunction partitionNum = partition -> {
    +                try {
    +                    return Integer.parseInt(partition);
    +                } catch (NumberFormatException e) {
    +                    throw new IllegalArgumentException("Invalid partition '" + partition + "' specified in topic arg '" + topicArg + "''");
    +                }
    +            };
    +
    +            String[] arr = topicArg.split(":");
    +
    +            if (arr.length != 2)
    +                throw new IllegalArgumentException("Invalid topic arg '" + topicArg + "', expected topic name and partitions");
    +
    +            String topic = arr[0];
    +            String partitions = arr[1];
    +
    +            return Arrays.stream(partitions.split(",")).
    +                map(partition -> new TopicPartition(topic, partitionNum.applyAsInt(partition)));
    +        }
    +
    +        private List parseTopicPartitionsToReset(List topicArgs) throws ExecutionException, InterruptedException {
    +            List topicsWithPartitions = new ArrayList<>();
    +            List topics = new ArrayList<>();
    +
    +            topicArgs.forEach(topicArg -> {
    +                if (topicArg.contains(":"))
    +                    topicsWithPartitions.add(topicArg);
    +                else
    +                    topics.add(topicArg);
    +            });
    +
    +            List specifiedPartitions = topicsWithPartitions.stream().flatMap(this::parseTopicsWithPartitions).collect(Collectors.toList());
    +
    +            List unspecifiedPartitions = new ArrayList<>();
    +
    +            if (!topics.isEmpty()) {
    +                Map descriptionMap = adminClient.describeTopics(
    +                    topics,
    +                    withTimeoutMs(new DescribeTopicsOptions())
    +                ).allTopicNames().get();
    +
    +                descriptionMap.forEach((topic, description) ->
    +                    description.partitions().forEach(tpInfo -> unspecifiedPartitions.add(new TopicPartition(topic, tpInfo.partition())))
    +                );
    +            }
    +
    +            specifiedPartitions.addAll(unspecifiedPartitions);
    +
    +            return specifiedPartitions;
    +        }
    +
    +        private Collection getPartitionsToReset(String groupId) throws ExecutionException, InterruptedException {
    +            if (opts.options.has(opts.allTopicsOpt)) {
    +                return getCommittedOffsets(groupId).keySet();
    +            } else if (opts.options.has(opts.topicOpt)) {
    +                List topics = opts.options.valuesOf(opts.topicOpt);
    +                return parseTopicPartitionsToReset(topics);
    +            } else {
    +                if (!opts.options.has(opts.resetFromFileOpt))
    +                    CommandLineUtils.printUsageAndExit(opts.parser, "One of the reset scopes should be defined: --all-topics, --topic.");
    +
    +                return Collections.emptyList();
    +            }
    +        }
    +
    +        private Map getCommittedOffsets(String groupId) {
    +            try {
    +                return adminClient.listConsumerGroupOffsets(
    +                    Collections.singletonMap(groupId, new ListConsumerGroupOffsetsSpec()),
    +                    withTimeoutMs(new ListConsumerGroupOffsetsOptions())
    +                ).partitionsToOffsetAndMetadata(groupId).get();
    +            } catch (InterruptedException | ExecutionException e) {
    +                throw new RuntimeException(e);
    +            }
    +        }
    +
    +        private Map> parseResetPlan(String resetPlanCsv) {
    +            ObjectReader csvReader = CsvUtils.readerFor(CsvUtils.CsvRecordNoGroup.class);
    +            String[] lines = resetPlanCsv.split("\n");
    +            boolean isSingleGroupQuery = opts.options.valuesOf(opts.groupOpt).size() == 1;
    +            boolean isOldCsvFormat = false;
    +            try {
    +                if (lines.length > 0) {
    +                    csvReader.readValue(lines[0], CsvUtils.CsvRecordNoGroup.class);
    +                    isOldCsvFormat = true;
    +                }
    +            } catch (IOException e) {
    +                e.printStackTrace();
    +                // Ignore.
    +            }
    +
    +            Map> dataMap = new HashMap<>();
    +
    +            try {
    +                // Single group CSV format: "topic,partition,offset"
    +                if (isSingleGroupQuery && isOldCsvFormat) {
    +                    String group = opts.options.valueOf(opts.groupOpt);
    +                    for (String line : lines) {
    +                        CsvUtils.CsvRecordNoGroup rec = csvReader.readValue(line, CsvUtils.CsvRecordNoGroup.class);
    +                        dataMap.computeIfAbsent(group, k -> new HashMap<>())
    +                            .put(new TopicPartition(rec.getTopic(), rec.getPartition()), new OffsetAndMetadata(rec.getOffset()));
    +                    }
    +                } else {
    +                    csvReader = CsvUtils.readerFor(CsvUtils.CsvRecordWithGroup.class);
    +                    for (String line : lines) {
    +                        CsvUtils.CsvRecordWithGroup rec = csvReader.readValue(line, CsvUtils.CsvRecordWithGroup.class);
    +                        dataMap.computeIfAbsent(rec.getGroup(), k -> new HashMap<>())
    +                            .put(new TopicPartition(rec.getTopic(), rec.getPartition()), new OffsetAndMetadata(rec.getOffset()));
    +                    }
    +                }
    +            } catch (IOException e) {
    +                throw new RuntimeException(e);
    +            }
    +
    +            return dataMap;
    +        }
    +
    +        @SuppressWarnings("CyclomaticComplexity")
    +        private Map prepareOffsetsToReset(String groupId, Collection partitionsToReset) {
    +            if (opts.options.has(opts.resetToOffsetOpt)) {
    +                long offset = opts.options.valueOf(opts.resetToOffsetOpt);
    +                return checkOffsetsRange(partitionsToReset.stream().collect(Collectors.toMap(Function.identity(), tp -> offset)))
    +                    .entrySet().stream().collect(Collectors.toMap(Entry::getKey, e -> new OffsetAndMetadata(e.getValue())));
    +            } else if (opts.options.has(opts.resetToEarliestOpt)) {
    +                Map logStartOffsets = getLogStartOffsets(partitionsToReset);
    +                return partitionsToReset.stream().collect(Collectors.toMap(Function.identity(), topicPartition -> {
    +                    LogOffsetResult logOffsetResult = logStartOffsets.get(topicPartition);
    +
    +                    if (!(logOffsetResult instanceof LogOffset)) {
    +                        ToolsUtils.printUsageAndExit(opts.parser, "Error getting starting offset of topic partition: " + topicPartition);
    +                        return null;
    +                    }
    +
    +                    return new OffsetAndMetadata(((LogOffset) logOffsetResult).value);
    +                }));
    +            } else if (opts.options.has(opts.resetToLatestOpt)) {
    +                Map logEndOffsets = getLogEndOffsets(partitionsToReset);
    +                return partitionsToReset.stream().collect(Collectors.toMap(Function.identity(), topicPartition -> {
    +                    LogOffsetResult logOffsetResult = logEndOffsets.get(topicPartition);
    +
    +                    if (!(logOffsetResult instanceof LogOffset)) {
    +                        ToolsUtils.printUsageAndExit(opts.parser, "Error getting ending offset of topic partition: " + topicPartition);
    +                        return null;
    +                    }
    +
    +                    return new OffsetAndMetadata(((LogOffset) logOffsetResult).value);
    +                }));
    +            } else if (opts.options.has(opts.resetShiftByOpt)) {
    +                Map currentCommittedOffsets = getCommittedOffsets(groupId);
    +                Map requestedOffsets = partitionsToReset.stream().collect(Collectors.toMap(Function.identity(), topicPartition -> {
    +                    long shiftBy = opts.options.valueOf(opts.resetShiftByOpt);
    +                    OffsetAndMetadata currentOffset = currentCommittedOffsets.get(topicPartition);
    +
    +                    if (currentOffset == null) {
    +                        throw new IllegalArgumentException("Cannot shift offset for partition " + topicPartition + " since there is no current committed offset");
    +                    }
    +
    +                    return currentOffset.offset() + shiftBy;
    +                }));
    +                return checkOffsetsRange(requestedOffsets).entrySet().stream()
    +                    .collect(Collectors.toMap(Entry::getKey, e -> new OffsetAndMetadata(e.getValue())));
    +            } else if (opts.options.has(opts.resetToDatetimeOpt)) {
    +                try {
    +                    long timestamp = Utils.getDateTime(opts.options.valueOf(opts.resetToDatetimeOpt));
    +                    Map logTimestampOffsets = getLogTimestampOffsets(partitionsToReset, timestamp);
    +                    return partitionsToReset.stream().collect(Collectors.toMap(Function.identity(), topicPartition -> {
    +                        LogOffsetResult logTimestampOffset = logTimestampOffsets.get(topicPartition);
    +
    +                        if (!(logTimestampOffset instanceof LogOffset)) {
    +                            ToolsUtils.printUsageAndExit(opts.parser, "Error getting offset by timestamp of topic partition: " + topicPartition);
    +                            return null;
    +                        }
    +
    +                        return new OffsetAndMetadata(((LogOffset) logTimestampOffset).value);
    +                    }));
    +                } catch (ParseException e) {
    +                    throw new RuntimeException(e);
    +                }
    +            } else if (opts.options.has(opts.resetByDurationOpt)) {
    +                String duration = opts.options.valueOf(opts.resetByDurationOpt);
    +                Duration durationParsed = Duration.parse(duration);
    +                Instant now = Instant.now();
    +                durationParsed.negated().addTo(now);
    +                long timestamp = now.minus(durationParsed).toEpochMilli();
    +                Map logTimestampOffsets = getLogTimestampOffsets(partitionsToReset, timestamp);
    +                return partitionsToReset.stream().collect(Collectors.toMap(Function.identity(), topicPartition -> {
    +                    LogOffsetResult logTimestampOffset = logTimestampOffsets.get(topicPartition);
    +
    +                    if (!(logTimestampOffset instanceof LogOffset)) {
    +                        ToolsUtils.printUsageAndExit(opts.parser, "Error getting offset by timestamp of topic partition: " + topicPartition);
    +                        return null;
    +                    }
    +
    +                    return new OffsetAndMetadata(((LogOffset) logTimestampOffset).value);
    +                }));
    +            } else if (resetPlanFromFile().isPresent()) {
    +                return resetPlanFromFile().map(resetPlan -> {
    +                    Map resetPlanForGroup = resetPlan.get(groupId);
    +
    +                    if (resetPlanForGroup == null) {
    +                        printError("No reset plan for group " + groupId + " found", Optional.empty());
    +                        return Collections.emptyMap();
    +                    }
    +
    +                    Map requestedOffsets = resetPlanForGroup.keySet().stream().collect(Collectors.toMap(
    +                        Function.identity(),
    +                        topicPartition -> resetPlanForGroup.get(topicPartition).offset()));
    +
    +                    return checkOffsetsRange(requestedOffsets).entrySet().stream()
    +                        .collect(Collectors.toMap(Entry::getKey, e -> new OffsetAndMetadata(e.getValue())));
    +                }).orElseGet(Collections::emptyMap);
    +            } else if (opts.options.has(opts.resetToCurrentOpt)) {
    +                Map currentCommittedOffsets = getCommittedOffsets(groupId);
    +                Collection partitionsToResetWithCommittedOffset = new ArrayList<>();
    +                Collection partitionsToResetWithoutCommittedOffset = new ArrayList<>();
    +
    +                for (TopicPartition topicPartition : partitionsToReset) {
    +                    if (currentCommittedOffsets.containsKey(topicPartition))
    +                        partitionsToResetWithCommittedOffset.add(topicPartition);
    +                    else
    +                        partitionsToResetWithoutCommittedOffset.add(topicPartition);
    +                }
    +
    +                Map preparedOffsetsForPartitionsWithCommittedOffset = partitionsToResetWithCommittedOffset.stream()
    +                    .collect(Collectors.toMap(Function.identity(), topicPartition -> {
    +                        OffsetAndMetadata committedOffset = currentCommittedOffsets.get(topicPartition);
    +
    +                        if (committedOffset == null) {
    +                            throw new IllegalStateException("Expected a valid current offset for topic partition: " + topicPartition);
    +                        }
    +
    +                        return new OffsetAndMetadata(committedOffset.offset());
    +                    }));
    +
    +                Map preparedOffsetsForPartitionsWithoutCommittedOffset = getLogEndOffsets(partitionsToResetWithoutCommittedOffset)
    +                    .entrySet().stream().collect(Collectors.toMap(Entry::getKey, e -> {
    +                        if (!(e.getValue() instanceof LogOffset)) {
    +                            ToolsUtils.printUsageAndExit(opts.parser, "Error getting ending offset of topic partition: " + e.getKey());
    +                            return null;
    +                        }
    +
    +                        return new OffsetAndMetadata(((LogOffset) e.getValue()).value);
    +                    }));
    +
    +                preparedOffsetsForPartitionsWithCommittedOffset.putAll(preparedOffsetsForPartitionsWithoutCommittedOffset);
    +
    +                return preparedOffsetsForPartitionsWithCommittedOffset;
    +            }
    +
    +            ToolsUtils.printUsageAndExit(opts.parser, String.format("Option '%s' requires one of the following scenarios: %s", opts.resetOffsetsOpt, opts.allResetOffsetScenarioOpts));
    +            return null;
    +        }
    +
    +        private Map checkOffsetsRange(Map requestedOffsets) {
    +            Map logStartOffsets = getLogStartOffsets(requestedOffsets.keySet());
    +            Map logEndOffsets = getLogEndOffsets(requestedOffsets.keySet());
    +
    +            Map res = new HashMap<>();
    +
    +            requestedOffsets.forEach((topicPartition, offset) -> {
    +                LogOffsetResult logEndOffset = logEndOffsets.get(topicPartition);
    +
    +                if (logEndOffset != null) {
    +                    if (logEndOffset instanceof LogOffset && offset > ((LogOffset) logEndOffset).value) {
    +                        long endOffset = ((LogOffset) logEndOffset).value;
    +                        LOGGER.warn("New offset (" + offset + ") is higher than latest offset for topic partition " + topicPartition + ". Value will be set to " + endOffset);
    +                        res.put(topicPartition, endOffset);
    +                    } else {
    +                        LogOffsetResult logStartOffset = logStartOffsets.get(topicPartition);
    +
    +                        if (logStartOffset instanceof LogOffset && offset < ((LogOffset) logStartOffset).value) {
    +                            long startOffset = ((LogOffset) logStartOffset).value;
    +                            LOGGER.warn("New offset (" + offset + ") is lower than earliest offset for topic partition " + topicPartition + ". Value will be set to " + startOffset);
    +                            res.put(topicPartition, startOffset);
    +                        } else
    +                            res.put(topicPartition, offset);
    +                    }
    +                } else {
    +                    // the control should not reach here
    +                    throw new IllegalStateException("Unexpected non-existing offset value for topic partition " + topicPartition);
    +                }
    +            });
    +
    +            return res;
    +        }
    +
    +        String exportOffsetsToCsv(Map> assignments) {
    +            boolean isSingleGroupQuery = opts.options.valuesOf(opts.groupOpt).size() == 1;
    +            ObjectWriter csvWriter = isSingleGroupQuery
    +                ? CsvUtils.writerFor(CsvUtils.CsvRecordNoGroup.class)
    +                : CsvUtils.writerFor(CsvUtils.CsvRecordWithGroup.class);
    +
    +            return Utils.mkString(assignments.entrySet().stream().flatMap(e -> {
    +                String groupId = e.getKey();
    +                Map partitionInfo = e.getValue();
    +
    +                return partitionInfo.entrySet().stream().map(e1 -> {
    +                    TopicPartition k = e1.getKey();
    +                    OffsetAndMetadata v = e1.getValue();
    +                    Object csvRecord = isSingleGroupQuery
    +                        ? new CsvUtils.CsvRecordNoGroup(k.topic(), k.partition(), v.offset())
    +                        : new CsvUtils.CsvRecordWithGroup(groupId, k.topic(), k.partition(), v.offset());
    +
    +                    try {
    +                        return csvWriter.writeValueAsString(csvRecord);
    +                    } catch (JsonProcessingException err) {
    +                        throw new RuntimeException(err);
    +                    }
    +                });
    +            }), "", "", "");
    +        }
    +
    +        Map deleteGroups() {
    +            List groupIds = opts.options.has(opts.allGroupsOpt)
    +                ? listConsumerGroups()
    +                : opts.options.valuesOf(opts.groupOpt);
    +
    +            Map> groupsToDelete = adminClient.deleteConsumerGroups(
    +                groupIds,
    +                withTimeoutMs(new DeleteConsumerGroupsOptions())
    +            ).deletedGroups();
    +
    +            Map success = new HashMap<>();
    +            Map failed = new HashMap<>();
    +
    +            groupsToDelete.forEach((g, f) -> {
    +                try {
    +                    f.get();
    +                    success.put(g, null);
    +                } catch (ExecutionException | InterruptedException e) {
    +                    failed.put(g, e);
    +                }
    +            });
    +
    +            if (failed.isEmpty())
    +                System.out.println("Deletion of requested consumer groups (" + Utils.mkString(success.keySet().stream(), "'", "'", "', '") + ") was successful.");
    +            else {
    +                printError("Deletion of some consumer groups failed:", Optional.empty());
    +                failed.forEach((group, error) -> System.out.println("* Group '" + group + "' could not be deleted due to: " + error));
    +
    +                if (!success.isEmpty())
    +                    System.out.println("\nThese consumer groups were deleted successfully: " + Utils.mkString(success.keySet().stream(), "'", "', '", "'"));
    +            }
    +
    +            failed.putAll(success);
    +
    +            return failed;
    +        }
    +    }
    +
    +    interface LogOffsetResult { }
    +
    +    private static class LogOffset implements LogOffsetResult {
    +        final long value;
    +
    +        LogOffset(long value) {
    +            this.value = value;
    +        }
    +    }
    +
    +    private static class Unknown implements LogOffsetResult { }
    +
    +    private static class Ignore implements LogOffsetResult { }
    +}
    diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandOptions.java
    index 045d296444..121594be4a 100644
    --- a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandOptions.java
    +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandOptions.java
    @@ -31,89 +31,99 @@
     import static org.apache.kafka.tools.ToolsUtils.minus;
     
     public class ConsumerGroupCommandOptions extends CommandDefaultOptions {
    -    public static final Logger LOGGER = LoggerFactory.getLogger(ConsumerGroupCommandOptions.class);
    +    private static final Logger LOGGER = LoggerFactory.getLogger(ConsumerGroupCommandOptions.class);
     
    -    public static final String BOOTSTRAP_SERVER_DOC = "REQUIRED: The server(s) to connect to.";
    -    public static final String GROUP_DOC = "The consumer group we wish to act on.";
    -    public static final String TOPIC_DOC = "The topic whose consumer group information should be deleted or topic whose should be included in the reset offset process. " +
    +    private static final String BOOTSTRAP_SERVER_DOC = "REQUIRED: The server(s) to connect to.";
    +    private static final String GROUP_DOC = "The consumer group we wish to act on.";
    +    private static final String TOPIC_DOC = "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. " +
             "Reset-offsets also supports multiple topic inputs.";
    -    public static final String ALL_TOPICS_DOC = "Consider all topics assigned to a group in the `reset-offsets` process.";
    -    public static final String LIST_DOC = "List all consumer groups.";
    -    public static final String DESCRIBE_DOC = "Describe consumer group and list offset lag (number of messages not yet processed) related to given group.";
    -    public static final String ALL_GROUPS_DOC = "Apply to all consumer groups.";
    -    public static final String NL = System.lineSeparator();
    -    public static final String DELETE_DOC = "Pass in groups to delete topic partition offsets and ownership information " +
    +    private static final String ALL_TOPICS_DOC = "Consider all topics assigned to a group in the `reset-offsets` process.";
    +    private static final String LIST_DOC = "List all consumer groups.";
    +    private static final String DESCRIBE_DOC = "Describe consumer group and list offset lag (number of messages not yet processed) related to given group.";
    +    private static final String ALL_GROUPS_DOC = "Apply to all consumer groups.";
    +    private static final String NL = System.lineSeparator();
    +    private static final String DELETE_DOC = "Pass in groups to delete topic partition offsets and ownership information " +
             "over the entire consumer group. For instance --group g1 --group g2";
    -    public static final String TIMEOUT_MS_DOC = "The timeout that can be set for some use cases. For example, it can be used when describing the group " +
    +    private static final String TIMEOUT_MS_DOC = "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).";
    -    public static final String COMMAND_CONFIG_DOC = "Property file containing configs to be passed to Admin Client and Consumer.";
    -    public static final String RESET_OFFSETS_DOC = "Reset offsets of consumer group. Supports one consumer group at the time, and instances should be inactive" + NL +
    +    private static final String COMMAND_CONFIG_DOC = "Property file containing configs to be passed to Admin Client and Consumer.";
    +    private static final String RESET_OFFSETS_DOC = "Reset offsets of consumer group. Supports one consumer group at the time, and instances should be inactive" + NL +
             "Has 2 execution options: --dry-run (the default) to plan which offsets to reset, and --execute to update the offsets. " +
             "Additionally, the --export option is used to export the results to a CSV format." + NL +
             "You must choose one of the following reset specifications: --to-datetime, --by-duration, --to-earliest, " +
             "--to-latest, --shift-by, --from-file, --to-current, --to-offset." + NL +
             "To define the scope use --all-topics or --topic. One scope must be specified unless you use '--from-file'.";
    -    public static final String DRY_RUN_DOC = "Only show results without executing changes on Consumer Groups. Supported operations: reset-offsets.";
    -    public static final String EXECUTE_DOC = "Execute operation. Supported operations: reset-offsets.";
    -    public static final String EXPORT_DOC = "Export operation execution to a CSV file. Supported operations: reset-offsets.";
    -    public static final String RESET_TO_OFFSET_DOC = "Reset offsets to a specific offset.";
    -    public static final String RESET_FROM_FILE_DOC = "Reset offsets to values defined in CSV file.";
    -    public static final String RESET_TO_DATETIME_DOC = "Reset offsets to offset from datetime. Format: 'YYYY-MM-DDTHH:mm:SS.sss'";
    -    public static final String RESET_BY_DURATION_DOC = "Reset offsets to offset by duration from current timestamp. Format: 'PnDTnHnMnS'";
    -    public static final String RESET_TO_EARLIEST_DOC = "Reset offsets to earliest offset.";
    -    public static final String RESET_TO_LATEST_DOC = "Reset offsets to latest offset.";
    -    public static final String RESET_TO_CURRENT_DOC = "Reset offsets to current offset.";
    -    public static final String RESET_SHIFT_BY_DOC = "Reset offsets shifting current offset by 'n', where 'n' can be positive or negative.";
    -    public static final String MEMBERS_DOC = "Describe members of the group. This option may be used with '--describe' and '--bootstrap-server' options only." + NL +
    +    private static final String DRY_RUN_DOC = "Only show results without executing changes on Consumer Groups. Supported operations: reset-offsets.";
    +    private static final String EXECUTE_DOC = "Execute operation. Supported operations: reset-offsets.";
    +    private static final String EXPORT_DOC = "Export operation execution to a CSV file. Supported operations: reset-offsets.";
    +    private static final String RESET_TO_OFFSET_DOC = "Reset offsets to a specific offset.";
    +    private static final String RESET_FROM_FILE_DOC = "Reset offsets to values defined in CSV file.";
    +    private static final String RESET_TO_DATETIME_DOC = "Reset offsets to offset from datetime. Format: 'YYYY-MM-DDTHH:mm:SS.sss'";
    +    private static final String RESET_BY_DURATION_DOC = "Reset offsets to offset by duration from current timestamp. Format: 'PnDTnHnMnS'";
    +    private static final String RESET_TO_EARLIEST_DOC = "Reset offsets to earliest offset.";
    +    private static final String RESET_TO_LATEST_DOC = "Reset offsets to latest offset.";
    +    private static final String RESET_TO_CURRENT_DOC = "Reset offsets to current offset.";
    +    private static final String RESET_SHIFT_BY_DOC = "Reset offsets shifting current offset by 'n', where 'n' can be positive or negative.";
    +    private static final String MEMBERS_DOC = "Describe members of the group. This option may be used with '--describe' and '--bootstrap-server' options only." + NL +
             "Example: --bootstrap-server localhost:9092 --describe --group group1 --members";
    -    public static final String VERBOSE_DOC = "Provide additional information, if any, when describing the group. This option may be used " +
    +    private static final String VERBOSE_DOC = "Provide additional information, if any, when describing the group. This option may be used " +
             "with '--offsets'/'--members'/'--state' and '--bootstrap-server' options only." + NL + "Example: --bootstrap-server localhost:9092 --describe --group group1 --members --verbose";
    -    public static final String OFFSETS_DOC = "Describe the group and list all topic partitions in the group along with their offset lag. " +
    +    private static final String OFFSETS_DOC = "Describe the group and list all topic partitions in the group along with their offset lag. " +
             "This is the default sub-action of and may be used with '--describe' and '--bootstrap-server' options only." + NL +
             "Example: --bootstrap-server localhost:9092 --describe --group group1 --offsets";
    -    public static final String STATE_DOC = "When specified with '--describe', includes the state of the group." + NL +
    +    private static final String STATE_DOC = "When specified with '--describe', includes the state of the group." + NL +
             "Example: --bootstrap-server localhost:9092 --describe --group group1 --state" + NL +
             "When specified with '--list', it displays the state of all groups. It can also be used to list groups with specific states." + NL +
             "Example: --bootstrap-server localhost:9092 --list --state stable,empty" + NL +
             "This option may be used with '--describe', '--list' and '--bootstrap-server' options only.";
    -    public static final String DELETE_OFFSETS_DOC = "Delete offsets of consumer group. Supports one consumer group at the time, and multiple topics.";
    +    private static final String TYPE_DOC = "When specified with '--list', it displays the types of all the groups. It can also be used to list groups with specific types." + NL +
    +        "Example: --bootstrap-server localhost:9092 --list --type classic,consumer" + NL +
    +        "This option may be used with the '--list' option only.";
    +    private static final String DELETE_OFFSETS_DOC = "Delete offsets of consumer group. Supports one consumer group at the time, and multiple topics.";
     
    -    public final OptionSpec bootstrapServerOpt;
    -    public final OptionSpec groupOpt;
    -    public final OptionSpec topicOpt;
    -    public final OptionSpec allTopicsOpt;
    -    public final OptionSpec listOpt;
    -    public final OptionSpec describeOpt;
    -    public final OptionSpec allGroupsOpt;
    -    public final OptionSpec deleteOpt;
    -    public final OptionSpec timeoutMsOpt;
    -    public final OptionSpec commandConfigOpt;
    -    public final OptionSpec resetOffsetsOpt;
    -    public final OptionSpec deleteOffsetsOpt;
    -    public final OptionSpec dryRunOpt;
    -    public final OptionSpec executeOpt;
    -    public final OptionSpec exportOpt;
    -    public final OptionSpec resetToOffsetOpt;
    -    public final OptionSpec resetFromFileOpt;
    -    public final OptionSpec resetToDatetimeOpt;
    -    public final OptionSpec resetByDurationOpt;
    -    public final OptionSpec resetToEarliestOpt;
    -    public final OptionSpec resetToLatestOpt;
    -    public final OptionSpec resetToCurrentOpt;
    -    public final OptionSpec resetShiftByOpt;
    -    public final OptionSpec membersOpt;
    -    public final OptionSpec verboseOpt;
    -    public final OptionSpec offsetsOpt;
    -    public final OptionSpec stateOpt;
    +    final OptionSpec bootstrapServerOpt;
    +    final OptionSpec groupOpt;
    +    final OptionSpec topicOpt;
    +    final OptionSpec allTopicsOpt;
    +    final OptionSpec listOpt;
    +    final OptionSpec describeOpt;
    +    final OptionSpec allGroupsOpt;
    +    final OptionSpec deleteOpt;
    +    final OptionSpec timeoutMsOpt;
    +    final OptionSpec commandConfigOpt;
    +    final OptionSpec resetOffsetsOpt;
    +    final OptionSpec deleteOffsetsOpt;
    +    final OptionSpec dryRunOpt;
    +    final OptionSpec executeOpt;
    +    final OptionSpec exportOpt;
    +    final OptionSpec resetToOffsetOpt;
    +    final OptionSpec resetFromFileOpt;
    +    final OptionSpec resetToDatetimeOpt;
    +    final OptionSpec resetByDurationOpt;
    +    final OptionSpec resetToEarliestOpt;
    +    final OptionSpec resetToLatestOpt;
    +    final OptionSpec resetToCurrentOpt;
    +    final OptionSpec resetShiftByOpt;
    +    final OptionSpec membersOpt;
    +    final OptionSpec verboseOpt;
    +    final OptionSpec offsetsOpt;
    +    final OptionSpec stateOpt;
    +    final OptionSpec typeOpt;
     
    -    public final Set> allGroupSelectionScopeOpts;
    -    public final Set> allConsumerGroupLevelOpts;
    -    public final Set> allResetOffsetScenarioOpts;
    -    public final Set> allDeleteOffsetsOpts;
    +    final Set> allGroupSelectionScopeOpts;
    +    final Set> allConsumerGroupLevelOpts;
    +    final Set> allResetOffsetScenarioOpts;
    +    final Set> allDeleteOffsetsOpts;
     
    -    public ConsumerGroupCommandOptions(String[] args) {
    +    public static ConsumerGroupCommandOptions fromArgs(String[] args) {
    +        ConsumerGroupCommandOptions opts = new ConsumerGroupCommandOptions(args);
    +        opts.checkArgs();
    +        return opts;
    +    }
    +
    +    private ConsumerGroupCommandOptions(String[] args) {
             super(args);
     
             bootstrapServerOpt = parser.accepts("bootstrap-server", BOOTSTRAP_SERVER_DOC)
    @@ -180,6 +190,10 @@ public ConsumerGroupCommandOptions(String[] args) {
                 .availableIf(describeOpt, listOpt)
                 .withOptionalArg()
                 .ofType(String.class);
    +        typeOpt = parser.accepts("type", TYPE_DOC)
    +            .availableIf(listOpt)
    +            .withOptionalArg()
    +            .ofType(String.class);
     
             allGroupSelectionScopeOpts = new HashSet<>(Arrays.asList(groupOpt, allGroupsOpt));
             allConsumerGroupLevelOpts = new HashSet<>(Arrays.asList(listOpt, describeOpt, deleteOpt, resetOffsetsOpt));
    @@ -191,7 +205,9 @@ public ConsumerGroupCommandOptions(String[] args) {
         }
     
         @SuppressWarnings({"CyclomaticComplexity", "NPathComplexity"})
    -    public void checkArgs() {
    +    void checkArgs() {
    +        CommandLineUtils.maybePrintHelpOrVersion(this, "This tool helps to list all consumer groups, describe a consumer group, delete consumer group info, or reset consumer group offsets.");
    +
             CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt);
     
             if (options.has(describeOpt)) {
    diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/CsvUtils.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/CsvUtils.java
    new file mode 100644
    index 0000000000..60a8bb22be
    --- /dev/null
    +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/CsvUtils.java
    @@ -0,0 +1,157 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements. See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License. You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.kafka.tools.consumer.group;
    +
    +import com.fasterxml.jackson.annotation.JsonProperty;
    +import com.fasterxml.jackson.databind.ObjectReader;
    +import com.fasterxml.jackson.databind.ObjectWriter;
    +import com.fasterxml.jackson.dataformat.csv.CsvMapper;
    +import com.fasterxml.jackson.dataformat.csv.CsvSchema;
    +
    +public class CsvUtils {
    +    private final static CsvMapper MAPPER = new CsvMapper();
    +
    +    static ObjectReader readerFor(Class clazz) {
    +        return MAPPER.readerFor(clazz).with(getSchema(clazz));
    +    }
    +
    +    static ObjectWriter writerFor(Class clazz) {
    +        return MAPPER.writerFor(clazz).with(getSchema(clazz));
    +    }
    +
    +    private static CsvSchema getSchema(Class clazz) {
    +        String[] fields;
    +        if (CsvRecordWithGroup.class == clazz)
    +            fields = CsvRecordWithGroup.FIELDS;
    +        else if (CsvRecordNoGroup.class == clazz)
    +            fields = CsvRecordNoGroup.FIELDS;
    +        else
    +            throw new IllegalStateException("Unhandled class " + clazz);
    +
    +        return MAPPER.schemaFor(clazz).sortedBy(fields);
    +    }
    +
    +    public static class CsvRecordWithGroup {
    +        public static final String[] FIELDS = new String[] {"group", "topic", "partition", "offset"};
    +
    +        @JsonProperty
    +        private String group;
    +
    +        @JsonProperty
    +        private String topic;
    +
    +        @JsonProperty
    +        private int partition;
    +
    +        @JsonProperty
    +        private long offset;
    +
    +        /**
    +         * Required for jackson.
    +         */
    +        public CsvRecordWithGroup() {
    +        }
    +
    +        public CsvRecordWithGroup(String group, String topic, int partition, long offset) {
    +            this.group = group;
    +            this.topic = topic;
    +            this.partition = partition;
    +            this.offset = offset;
    +        }
    +
    +        public void setGroup(String group) {
    +            this.group = group;
    +        }
    +
    +        public String getGroup() {
    +            return group;
    +        }
    +
    +        public String getTopic() {
    +            return topic;
    +        }
    +
    +        public void setTopic(String topic) {
    +            this.topic = topic;
    +        }
    +
    +        public int getPartition() {
    +            return partition;
    +        }
    +
    +        public void setPartition(int partition) {
    +            this.partition = partition;
    +        }
    +
    +        public long getOffset() {
    +            return offset;
    +        }
    +
    +        public void setOffset(long offset) {
    +            this.offset = offset;
    +        }
    +    }
    +
    +    public static class CsvRecordNoGroup {
    +        public static final String[] FIELDS = new String[]{"topic", "partition", "offset"};
    +
    +        @JsonProperty
    +        private String topic;
    +
    +        @JsonProperty
    +        private int partition;
    +
    +        @JsonProperty
    +        private long offset;
    +
    +        /**
    +         * Required for jackson.
    +         */
    +        public CsvRecordNoGroup() {
    +        }
    +
    +        public CsvRecordNoGroup(String topic, int partition, long offset) {
    +            this.topic = topic;
    +            this.partition = partition;
    +            this.offset = offset;
    +        }
    +
    +        public String getTopic() {
    +            return topic;
    +        }
    +
    +        public void setTopic(String topic) {
    +            this.topic = topic;
    +        }
    +
    +        public int getPartition() {
    +            return partition;
    +        }
    +
    +        public void setPartition(int partition) {
    +            this.partition = partition;
    +        }
    +
    +        public long getOffset() {
    +            return offset;
    +        }
    +
    +        public void setOffset(long offset) {
    +            this.offset = offset;
    +        }
    +    }
    +}
    diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/GroupState.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/GroupState.java
    new file mode 100644
    index 0000000000..04a3b2eae1
    --- /dev/null
    +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/GroupState.java
    @@ -0,0 +1,35 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements. See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License. You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.kafka.tools.consumer.group;
    +
    +import org.apache.kafka.common.Node;
    +
    +class GroupState {
    +    final String group;
    +    final Node coordinator;
    +    final String assignmentStrategy;
    +    final String state;
    +    final int numMembers;
    +
    +    GroupState(String group, Node coordinator, String assignmentStrategy, String state, int numMembers) {
    +        this.group = group;
    +        this.coordinator = coordinator;
    +        this.assignmentStrategy = assignmentStrategy;
    +        this.state = state;
    +        this.numMembers = numMembers;
    +    }
    +}
    diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/MemberAssignmentState.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/MemberAssignmentState.java
    index 040cb1c741..9ac4642833 100644
    --- a/tools/src/main/java/org/apache/kafka/tools/consumer/group/MemberAssignmentState.java
    +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/MemberAssignmentState.java
    @@ -21,15 +21,15 @@
     import java.util.List;
     
     class MemberAssignmentState {
    -    public final String group;
    -    public final String consumerId;
    -    public final String host;
    -    public final String clientId;
    -    public final String groupInstanceId;
    -    public final int numPartitions;
    -    public final List assignment;
    +    final String group;
    +    final String consumerId;
    +    final String host;
    +    final String clientId;
    +    final String groupInstanceId;
    +    final int numPartitions;
    +    final List assignment;
     
    -    public MemberAssignmentState(String group, String consumerId, String host, String clientId, String groupInstanceId,
    +    MemberAssignmentState(String group, String consumerId, String host, String clientId, String groupInstanceId,
                                      int numPartitions, List assignment) {
             this.group = group;
             this.consumerId = consumerId;
    diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/PartitionAssignmentState.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/PartitionAssignmentState.java
    index 396032f0a0..9e45f2b05e 100644
    --- a/tools/src/main/java/org/apache/kafka/tools/consumer/group/PartitionAssignmentState.java
    +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/PartitionAssignmentState.java
    @@ -19,25 +19,23 @@
     import org.apache.kafka.common.Node;
     
     import java.util.Optional;
    -import java.util.OptionalInt;
    -import java.util.OptionalLong;
     
     class PartitionAssignmentState {
    -    public final String group;
    -    public final Optional coordinator;
    -    public final Optional topic;
    -    public final OptionalInt partition;
    -    public final OptionalLong offset;
    -    public final OptionalLong lag;
    -    public final Optional consumerId;
    -    public final Optional host;
    -    public final Optional clientId;
    -    public final OptionalLong logEndOffset;
    +    final String group;
    +    final Optional coordinator;
    +    final Optional topic;
    +    final Optional partition;
    +    final Optional offset;
    +    final Optional lag;
    +    final Optional consumerId;
    +    final Optional host;
    +    final Optional clientId;
    +    final Optional logEndOffset;
     
    -    public PartitionAssignmentState(String group, Optional coordinator, Optional topic,
    -                                    OptionalInt partition, OptionalLong offset, OptionalLong lag,
    +    PartitionAssignmentState(String group, Optional coordinator, Optional topic,
    +                                    Optional partition, Optional offset, Optional lag,
                                         Optional consumerId, Optional host, Optional clientId,
    -                                    OptionalLong logEndOffset) {
    +                                    Optional logEndOffset) {
             this.group = group;
             this.coordinator = coordinator;
             this.topic = topic;
    diff --git a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java
    index 3623346980..47c4a8234a 100644
    --- a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java
    +++ b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java
    @@ -49,9 +49,9 @@
     import org.apache.kafka.server.util.json.JsonValue;
     import org.apache.kafka.tools.TerseException;
     import org.apache.kafka.tools.ToolsUtils;
    -import org.apache.kafka.tools.Tuple2;
     
     import java.io.IOException;
    +import java.util.AbstractMap.SimpleImmutableEntry;
     import java.util.ArrayList;
     import java.util.Arrays;
     import java.util.Collection;
    @@ -62,6 +62,7 @@
     import java.util.LinkedHashMap;
     import java.util.List;
     import java.util.Map;
    +import java.util.Map.Entry;
     import java.util.Objects;
     import java.util.Optional;
     import java.util.Properties;
    @@ -203,20 +204,20 @@ static VerifyAssignmentResult verifyAssignment(Admin adminClient,
                                                        String jsonString,
                                                        Boolean preserveThrottles
         ) throws ExecutionException, InterruptedException, JsonProcessingException {
    -        Tuple2>>, Map> t0 = parsePartitionReassignmentData(jsonString);
    +        Entry>>, Map> t0 = parsePartitionReassignmentData(jsonString);
     
    -        List>> targetParts = t0.v1;
    -        Map targetLogDirs = t0.v2;
    +        List>> targetParts = t0.getKey();
    +        Map targetLogDirs = t0.getValue();
     
    -        Tuple2, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
    +        Entry, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
     
    -        Map partStates = t1.v1;
    -        Boolean partsOngoing = t1.v2;
    +        Map partStates = t1.getKey();
    +        Boolean partsOngoing = t1.getValue();
     
    -        Tuple2, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
    +        Entry, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
     
    -        Map moveStates = t2.v1;
    -        Boolean movesOngoing = t2.v2;
    +        Map moveStates = t2.getKey();
    +        Boolean movesOngoing = t2.getValue();
     
             if (!partsOngoing && !movesOngoing && !preserveThrottles) {
                 // If the partition assignments and replica assignments are done, clear any throttles
    @@ -240,11 +241,11 @@ static VerifyAssignmentResult verifyAssignment(Admin adminClient,
          *                              reassignments (including reassignments not described
          *                              in the JSON file.)
          */
    -    private static Tuple2, Boolean> verifyPartitionAssignments(Admin adminClient,
    -                                                                                                               List>> targets
    +    private static Entry, Boolean> verifyPartitionAssignments(Admin adminClient,
    +                                                                                                               List>> targets
         ) throws ExecutionException, InterruptedException {
    -        Tuple2, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
    -        System.out.println(partitionReassignmentStatesToString(t0.v1));
    +        Entry, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
    +        System.out.println(partitionReassignmentStatesToString(t0.getKey()));
             return t0;
         }
     
    @@ -301,27 +302,27 @@ static String partitionReassignmentStatesToString(Map, Boolean> findPartitionReassignmentStates(Admin adminClient,
    -                                                                                                            List, Boolean> findPartitionReassignmentStates(Admin adminClient,
    +                                                                                                            List>> targetReassignments
         ) throws ExecutionException, InterruptedException {
             Map currentReassignments = adminClient.
                 listPartitionReassignments().reassignments().get();
     
    -        List>> foundReassignments = new ArrayList<>();
    -        List>> notFoundReassignments = new ArrayList<>();
    +        List>> foundReassignments = new ArrayList<>();
    +        List>> notFoundReassignments = new ArrayList<>();
     
             targetReassignments.forEach(reassignment -> {
    -            if (currentReassignments.containsKey(reassignment.v1))
    +            if (currentReassignments.containsKey(reassignment.getKey()))
                     foundReassignments.add(reassignment);
                 else
                     notFoundReassignments.add(reassignment);
             });
     
    -        List> foundResults = foundReassignments.stream().map(e -> {
    -            TopicPartition part = e.v1;
    -            List targetReplicas = e.v2;
    -            return new Tuple2<>(part,
    +        List> foundResults = foundReassignments.stream().map(e -> {
    +            TopicPartition part = e.getKey();
    +            List targetReplicas = e.getValue();
    +            return new SimpleImmutableEntry<>(part,
                     new PartitionReassignmentState(
                         currentReassignments.get(part).replicas(),
                         targetReplicas,
    @@ -329,7 +330,7 @@ static Tuple2, Boolean> findPart
             }).collect(Collectors.toList());
     
             Set topicNamesToLookUp = notFoundReassignments.stream()
    -            .map(e -> e.v1)
    +            .map(e -> e.getKey())
                 .filter(part -> !currentReassignments.containsKey(part))
                 .map(TopicPartition::topic)
                 .collect(Collectors.toSet());
    @@ -337,28 +338,28 @@ static Tuple2, Boolean> findPart
             Map> topicDescriptions = adminClient.
                 describeTopics(topicNamesToLookUp).topicNameValues();
     
    -        List> notFoundResults = new ArrayList<>();
    -        for (Tuple2> e : notFoundReassignments) {
    -            TopicPartition part = e.v1;
    -            List targetReplicas = e.v2;
    +        List> notFoundResults = new ArrayList<>();
    +        for (Entry> e : notFoundReassignments) {
    +            TopicPartition part = e.getKey();
    +            List targetReplicas = e.getValue();
     
                 if (currentReassignments.containsKey(part)) {
                     PartitionReassignment reassignment = currentReassignments.get(part);
    -                notFoundResults.add(new Tuple2<>(part, new PartitionReassignmentState(
    +                notFoundResults.add(new SimpleImmutableEntry<>(part, new PartitionReassignmentState(
                         reassignment.replicas(),
                         targetReplicas,
                         false)));
                 } else {
    -                notFoundResults.add(new Tuple2<>(part, topicDescriptionFutureToState(part.partition(),
    +                notFoundResults.add(new SimpleImmutableEntry<>(part, topicDescriptionFutureToState(part.partition(),
                         topicDescriptions.get(part.topic()), targetReplicas)));
                 }
             }
     
             Map allResults = new HashMap<>();
    -        foundResults.forEach(e -> allResults.put(e.v1, e.v2));
    -        notFoundResults.forEach(e -> allResults.put(e.v1, e.v2));
    +        foundResults.forEach(e -> allResults.put(e.getKey(), e.getValue()));
    +        notFoundResults.forEach(e -> allResults.put(e.getKey(), e.getValue()));
     
    -        return new Tuple2<>(allResults, !currentReassignments.isEmpty());
    +        return new SimpleImmutableEntry<>(allResults, !currentReassignments.isEmpty());
         }
     
         private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
    @@ -396,12 +397,12 @@ private static PartitionReassignmentState topicDescriptionFutureToState(int part
          *                              reassignments. (We don't have an efficient API that
          *                              returns all ongoing replica reassignments.)
          */
    -    private static Tuple2, Boolean> verifyReplicaMoves(Admin adminClient,
    +    private static Entry, Boolean> verifyReplicaMoves(Admin adminClient,
                                                                                                        Map targetReassignments
         ) throws ExecutionException, InterruptedException {
             Map moveStates = findLogDirMoveStates(adminClient, targetReassignments);
             System.out.println(replicaMoveStatesToString(moveStates));
    -        return new Tuple2<>(moveStates, !moveStates.values().stream().allMatch(LogDirMoveState::done));
    +        return new SimpleImmutableEntry<>(moveStates, !moveStates.values().stream().allMatch(LogDirMoveState::done));
         }
     
         /**
    @@ -420,7 +421,7 @@ static Map findLogDirMoveStates(Admin ad
             Map replicaLogDirInfos = adminClient
                 .describeReplicaLogDirs(targetMoves.keySet()).all().get();
     
    -        return targetMoves.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> {
    +        return targetMoves.entrySet().stream().collect(Collectors.toMap(Entry::getKey, e -> {
                 TopicPartitionReplica replica = e.getKey();
                 String targetLogDir = e.getValue();
     
    @@ -492,16 +493,16 @@ static String replicaMoveStatesToString(Map>> targetParts
    +                                          List>> targetParts
         ) throws ExecutionException, InterruptedException {
             Set brokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
    -        targetParts.forEach(t -> brokers.addAll(t.v2));
    +        targetParts.forEach(t -> brokers.addAll(t.getValue()));
     
             System.out.printf("Clearing broker-level throttles on broker%s %s%n",
                 brokers.size() == 1 ? "" : "s", Utils.join(brokers, ","));
             clearBrokerLevelThrottles(adminClient, brokers);
     
    -        Set topics = targetParts.stream().map(t -> t.v1.topic()).collect(Collectors.toSet());
    +        Set topics = targetParts.stream().map(t -> t.getKey().topic()).collect(Collectors.toSet());
             System.out.printf("Clearing topic-level throttles on topic%s %s%n",
                 topics.size() == 1 ? "" : "s", Utils.join(topics, ","));
             clearTopicLevelThrottles(adminClient, topics);
    @@ -550,15 +551,15 @@ private static void clearTopicLevelThrottles(Admin adminClient, Set topi
          * @return                      A tuple containing the proposed assignment and the
          *                              current assignment.
          */
    -    public static Tuple2>, Map>> generateAssignment(Admin adminClient,
    +    public static Entry>, Map>> generateAssignment(Admin adminClient,
                                                                                                                  String reassignmentJson,
                                                                                                                  String brokerListString,
                                                                                                                  Boolean enableRackAwareness
         ) throws ExecutionException, InterruptedException, JsonProcessingException {
    -        Tuple2, List> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
    +        Entry, List> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
     
    -        List brokersToReassign = t0.v1;
    -        List topicsToReassign = t0.v2;
    +        List brokersToReassign = t0.getKey();
    +        List topicsToReassign = t0.getValue();
     
             Map> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
             List brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
    @@ -567,7 +568,7 @@ public static Tuple2>, Map(proposedAssignments, currentAssignments);
    +        return new SimpleImmutableEntry<>(proposedAssignments, currentAssignments);
         }
     
         /**
    @@ -580,8 +581,8 @@ public static Tuple2>, Map> calculateAssignment(Map> currentAssignment,
                                                                               List brokerMetadatas) {
    -        Map>>> groupedByTopic = new HashMap<>();
    -        for (Map.Entry> e : currentAssignment.entrySet())
    +        Map>>> groupedByTopic = new HashMap<>();
    +        for (Entry> e : currentAssignment.entrySet())
                 groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
             Map> proposedAssignments = new HashMap<>();
             groupedByTopic.forEach((topic, assignment) -> {
    @@ -598,7 +599,7 @@ private static Map describeTopics(Admin adminClient,
                                                                     Set topics) throws ExecutionException, InterruptedException {
             Map> futures = adminClient.describeTopics(topics).topicNameValues();
             Map res = new HashMap<>();
    -        for (Map.Entry> e : futures.entrySet()) {
    +        for (Entry> e : futures.entrySet()) {
                 String topicName = e.getKey();
                 KafkaFuture topicDescriptionFuture = e.getValue();
                 try {
    @@ -695,7 +696,7 @@ static List getBrokerMetadata(Admin adminClient, List b
          *
          * @return                       A tuple of brokers to reassign, topics to reassign
          */
    -    static Tuple2, List> parseGenerateAssignmentArgs(String reassignmentJson,
    +    static Entry, List> parseGenerateAssignmentArgs(String reassignmentJson,
                                                                                String brokerList) throws JsonMappingException {
             List brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
             Set duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
    @@ -706,7 +707,7 @@ static Tuple2, List> parseGenerateAssignmentArgs(String re
             if (!duplicateTopicsToReassign.isEmpty())
                 throw new AdminCommandFailedException(String.format("List of topics to reassign contains duplicate entries: %s",
                     duplicateTopicsToReassign));
    -        return new Tuple2<>(brokerListToReassign, topicsToReassign);
    +        return new SimpleImmutableEntry<>(brokerListToReassign, topicsToReassign);
         }
     
         /**
    @@ -731,10 +732,10 @@ public static void executeAssignment(Admin adminClient,
                                       Long timeoutMs,
                                       Time time
         ) throws ExecutionException, InterruptedException, JsonProcessingException, TerseException {
    -        Tuple2>, Map> t0 = parseExecuteAssignmentArgs(reassignmentJson);
    +        Entry>, Map> t0 = parseExecuteAssignmentArgs(reassignmentJson);
     
    -        Map> proposedParts = t0.v1;
    -        Map proposedReplicas = t0.v2;
    +        Map> proposedParts = t0.getKey();
    +        Map proposedReplicas = t0.getValue();
             Map currentReassignments = adminClient.
                 listPartitionReassignments().reassignments().get();
             // If there is an existing assignment, check for --additional before proceeding.
    @@ -901,7 +902,7 @@ static String currentPartitionReplicaAssignmentToString(Map> currentParts) throws JsonProcessingException {
             Map> partitionsToBeReassigned = currentParts.entrySet().stream()
                 .filter(e -> proposedParts.containsKey(e.getKey()))
    -            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
    +            .collect(Collectors.toMap(Entry::getKey, Entry::getValue));
     
             return String.format("Current partition replica assignment%n%n%s%n%nSave this to use as the %s",
                 formatAsReassignmentJson(partitionsToBeReassigned, Collections.emptyMap()),
    @@ -921,7 +922,7 @@ static Map alterPartitionReassignments(Admin adminCli
             reassignments.forEach((part, replicas) -> args.put(part, Optional.of(new NewPartitionReassignment(replicas))));
             Map> results = adminClient.alterPartitionReassignments(args).values();
             Map errors = new HashMap<>();
    -        for (Map.Entry> e :  results.entrySet()) {
    +        for (Entry> e :  results.entrySet()) {
                 try {
                     e.getValue().get();
                 } catch (ExecutionException t) {
    @@ -945,7 +946,7 @@ static Map cancelPartitionReassignments(Admin adminCl
     
             Map> results = adminClient.alterPartitionReassignments(args).values();
             Map errors = new HashMap<>();
    -        for (Map.Entry> e :  results.entrySet()) {
    +        for (Entry> e :  results.entrySet()) {
                 try {
                     e.getValue().get();
                 } catch (ExecutionException t) {
    @@ -993,7 +994,7 @@ static Map> calculateProposedMoveMap(Map> proposedParts,
                                                                                  Map> currentParts) {
             Map> moveMap = calculateCurrentMoveMap(currentReassignments);
    -        for (Map.Entry> e : proposedParts.entrySet()) {
    +        for (Entry> e : proposedParts.entrySet()) {
                 TopicPartition part = e.getKey();
                 List replicas = e.getValue();
                 Map partMoves = moveMap.computeIfAbsent(part.topic(), k -> new HashMap<>());
    @@ -1178,39 +1179,39 @@ static void modifyLogDirThrottle(Admin admin,
          * @return                  A tuple of the partitions to be reassigned and the replicas
          *                          to be reassigned.
          */
    -    static Tuple2>, Map> parseExecuteAssignmentArgs(
    +    static Entry>, Map> parseExecuteAssignmentArgs(
             String reassignmentJson
         ) throws JsonProcessingException {
    -        Tuple2>>, Map> t0 = parsePartitionReassignmentData(reassignmentJson);
    +        Entry>>, Map> t0 = parsePartitionReassignmentData(reassignmentJson);
     
    -        List>> partitionsToBeReassigned = t0.v1;
    -        Map replicaAssignment = t0.v2;
    +        List>> partitionsToBeReassigned = t0.getKey();
    +        Map replicaAssignment = t0.getValue();
     
             if (partitionsToBeReassigned.isEmpty())
                 throw new AdminCommandFailedException("Partition reassignment list cannot be empty");
    -        if (partitionsToBeReassigned.stream().anyMatch(t -> t.v2.isEmpty())) {
    +        if (partitionsToBeReassigned.stream().anyMatch(t -> t.getValue().isEmpty())) {
                 throw new AdminCommandFailedException("Partition replica list cannot be empty");
             }
    -        Set duplicateReassignedPartitions = ToolsUtils.duplicates(partitionsToBeReassigned.stream().map(t -> t.v1).collect(Collectors.toList()));
    +        Set duplicateReassignedPartitions = ToolsUtils.duplicates(partitionsToBeReassigned.stream().map(t -> t.getKey()).collect(Collectors.toList()));
             if (!duplicateReassignedPartitions.isEmpty()) {
                 throw new AdminCommandFailedException(String.format(
                     "Partition reassignment contains duplicate topic partitions: %s",
                     duplicateReassignedPartitions.stream().map(Object::toString).collect(Collectors.joining(",")))
                 );
             }
    -        List>> duplicateEntries = partitionsToBeReassigned.stream()
    -            .map(t -> new Tuple2<>(t.v1, ToolsUtils.duplicates(t.v2)))
    -            .filter(t -> !t.v2.isEmpty())
    +        List>> duplicateEntries = partitionsToBeReassigned.stream()
    +            .map(t -> new SimpleImmutableEntry<>(t.getKey(), ToolsUtils.duplicates(t.getValue())))
    +            .filter(t -> !t.getValue().isEmpty())
                 .collect(Collectors.toList());
             if (!duplicateEntries.isEmpty()) {
                 String duplicatesMsg = duplicateEntries.stream().map(t ->
                     String.format("%s contains multiple entries for %s",
    -                    t.v1,
    -                    t.v2.stream().map(Object::toString).collect(Collectors.joining(",")))
    +                    t.getKey(),
    +                    t.getValue().stream().map(Object::toString).collect(Collectors.joining(",")))
                 ).collect(Collectors.joining(". "));
                 throw new AdminCommandFailedException(String.format("Partition replica lists may not contain duplicate entries: %s", duplicatesMsg));
             }
    -        return new Tuple2<>(partitionsToBeReassigned.stream().collect(Collectors.toMap(t -> t.v1, t -> t.v2)), replicaAssignment);
    +        return new SimpleImmutableEntry<>(partitionsToBeReassigned.stream().collect(Collectors.toMap(t -> t.getKey(), t -> t.getValue())), replicaAssignment);
         }
     
         /**
    @@ -1226,17 +1227,17 @@ static Tuple2>, Map, Set> cancelAssignment(Admin adminClient,
    +    static Entry, Set> cancelAssignment(Admin adminClient,
                                                                                         String jsonString,
                                                                                         Boolean preserveThrottles,
                                                                                         Long timeoutMs,
                                                                                         Time time
         ) throws ExecutionException, InterruptedException, JsonProcessingException, TerseException {
    -        Tuple2>>, Map> t0 = parsePartitionReassignmentData(jsonString);
    +        Entry>>, Map> t0 = parsePartitionReassignmentData(jsonString);
     
    -        List>> targetParts = t0.v1;
    -        Map targetReplicas = t0.v2;
    -        Set targetPartsSet = targetParts.stream().map(t -> t.v1).collect(Collectors.toSet());
    +        List>> targetParts = t0.getKey();
    +        Map targetReplicas = t0.getValue();
    +        Set targetPartsSet = targetParts.stream().map(t -> t.getKey()).collect(Collectors.toSet());
             Set curReassigningParts = new HashSet<>();
             adminClient.listPartitionReassignments(targetPartsSet).reassignments().get().forEach((part, reassignment) -> {
                 if (reassignment.addingReplicas().isEmpty() || !reassignment.removingReplicas().isEmpty())
    @@ -1273,7 +1274,7 @@ static Tuple2, Set> cancelAssignment(
             if (!preserveThrottles) {
                 clearAllThrottles(adminClient, targetParts);
             }
    -        return new Tuple2<>(curReassigningParts, curMovingParts.keySet());
    +        return new SimpleImmutableEntry<>(curReassigningParts, curMovingParts.keySet());
         }
     
         public static String formatAsReassignmentJson(Map> partitionsToBeReassigned,
    @@ -1330,7 +1331,7 @@ private static List parseTopicsData(int version, JsonValue js) throws Js
             }
         }
     
    -    private static Tuple2>>, Map> parsePartitionReassignmentData(
    +    private static Entry>>, Map> parsePartitionReassignmentData(
             String jsonData
         ) throws JsonProcessingException {
             JsonValue js;
    @@ -1344,12 +1345,12 @@ private static Tuple2>>, Map>>, Map> parsePartitionReassignmentData(
    +    private static Entry>>, Map> parsePartitionReassignmentData(
             int version, JsonValue jsonData
         ) throws JsonMappingException {
             switch (version) {
                 case 1:
    -                List>> partitionAssignment = new ArrayList<>();
    +                List>> partitionAssignment = new ArrayList<>();
                     Map replicaAssignment = new HashMap<>();
     
                     Optional partitionsSeq = jsonData.asJsonObject().get("partitions");
    @@ -1369,7 +1370,7 @@ private static Tuple2>>, Map(new TopicPartition(topic, partition), newReplicas));
    +                        partitionAssignment.add(new SimpleImmutableEntry<>(new TopicPartition(topic, partition), newReplicas));
                             for (int i = 0; i < newLogDirs.size(); i++) {
                                 Integer replica = newReplicas.get(i);
                                 String logDir = newLogDirs.get(i);
    @@ -1382,7 +1383,7 @@ private static Tuple2>>, Map(partitionAssignment, replicaAssignment);
    +                return new SimpleImmutableEntry<>(partitionAssignment, replicaAssignment);
     
                 default:
                     throw new AdminOperationException("Not supported version field value " + version);
    @@ -1481,7 +1482,7 @@ static Set alterReplicaLogDirs(Admin adminClient,
             Set results = new HashSet<>();
             Map> values = adminClient.alterReplicaLogDirs(assignment).values();
     
    -        for (Map.Entry> e : values.entrySet()) {
    +        for (Entry> e : values.entrySet()) {
                 TopicPartitionReplica replica = e.getKey();
                 KafkaFuture future = e.getValue();
                 try {
    diff --git a/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java
    index b540c8ffc1..0f4e551a48 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java
    @@ -120,7 +120,11 @@ private static void executeAndAssertOutput(String json, String expOut, Admin adm
     class DeleteRecordsCommandUnitTest {
         @Test
         public void testOffsetFileNotExists() {
    -        assertThrows(IOException.class, () -> DeleteRecordsCommand.main(new String[]{
    +        assertThrows(IOException.class, () -> DeleteRecordsCommand.execute(new String[]{
    +            "--bootstrap-server", "localhost:9092",
    +            "--offset-json-file", "/not/existing/file"
    +        }, System.out));
    +        assertEquals(1, DeleteRecordsCommand.mainNoExit(new String[]{
                 "--bootstrap-server", "localhost:9092",
                 "--offset-json-file", "/not/existing/file"
             }));
    @@ -128,7 +132,12 @@ public void testOffsetFileNotExists() {
     
         @Test
         public void testCommandConfigNotExists() {
    -        assertThrows(NoSuchFileException.class, () -> DeleteRecordsCommand.main(new String[] {
    +        assertThrows(NoSuchFileException.class, () -> DeleteRecordsCommand.execute(new String[] {
    +            "--bootstrap-server", "localhost:9092",
    +            "--offset-json-file", "/not/existing/file",
    +            "--command-config", "/another/not/existing/file"
    +        }, System.out));
    +        assertEquals(1, DeleteRecordsCommand.mainNoExit(new String[] {
                 "--bootstrap-server", "localhost:9092",
                 "--offset-json-file", "/not/existing/file",
                 "--command-config", "/another/not/existing/file"
    diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java
    index 417bbe7116..dad41342b7 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java
    @@ -29,7 +29,9 @@
     import org.apache.kafka.clients.producer.ProducerConfig;
     import org.apache.kafka.clients.producer.ProducerRecord;
     import org.apache.kafka.common.serialization.StringSerializer;
    +import org.apache.kafka.common.utils.AppInfoParser;
     import org.apache.kafka.common.utils.Exit;
    +import org.junit.jupiter.api.BeforeEach;
     import org.junit.jupiter.api.Tag;
     import org.junit.jupiter.api.extension.ExtendWith;
     
    @@ -52,20 +54,24 @@ public class GetOffsetShellTest {
         private final int topicCount = 4;
         private final int offsetTopicPartitionCount = 4;
         private final ClusterInstance cluster;
    +    private final String topicName = "topic";
     
         public GetOffsetShellTest(ClusterInstance cluster) {
             this.cluster = cluster;
         }
     
         private String getTopicName(int i) {
    -        return "topic" + i;
    +        return topicName + i;
         }
     
    -    public void setUp() {
    +    @BeforeEach
    +    public void before() {
             cluster.config().serverProperties().put("auto.create.topics.enable", false);
             cluster.config().serverProperties().put("offsets.topic.replication.factor", "1");
             cluster.config().serverProperties().put("offsets.topic.num.partitions", String.valueOf(offsetTopicPartitionCount));
    +    }
     
    +    private void setUp() {
             try (Admin admin = Admin.create(cluster.config().adminClientProperties())) {
                 List topics = new ArrayList<>();
     
    @@ -312,6 +318,18 @@ public void testTopicPartitionsFlagWithPartitionsFlagCauseExit() {
             assertExitCodeIsOne("--topic-partitions", "__consumer_offsets", "--partitions", "0");
         }
     
    +    @ClusterTest
    +    public void testPrintHelp() {
    +        String out = ToolsTestUtils.captureStandardErr(() -> GetOffsetShell.mainNoExit("--help"));
    +        assertTrue(out.startsWith(GetOffsetShell.USAGE_TEXT));
    +    }
    +
    +    @ClusterTest
    +    public void testPrintVersion() {
    +        String out = ToolsTestUtils.captureStandardOut(() -> GetOffsetShell.mainNoExit("--version"));
    +        assertEquals(AppInfoParser.getVersion(), out);
    +    }
    +
         private void assertExitCodeIsOne(String... args) {
             final int[] exitStatus = new int[1];
     
    @@ -333,7 +351,7 @@ private void assertExitCodeIsOne(String... args) {
         }
     
         private List expectedOffsetsWithInternal() {
    -        List consOffsets = IntStream.range(0, offsetTopicPartitionCount + 1)
    +        List consOffsets = IntStream.range(0, offsetTopicPartitionCount)
                     .mapToObj(i -> new Row("__consumer_offsets", i, 0L))
                     .collect(Collectors.toList());
     
    diff --git a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandErrorTest.java b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandErrorTest.java
    index fef75bfd10..087621df7d 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandErrorTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandErrorTest.java
    @@ -22,6 +22,8 @@
     
     import java.time.Duration;
     
    +import static org.junit.jupiter.api.Assertions.assertEquals;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     import static org.junit.jupiter.api.Assertions.assertThrows;
     import static org.junit.jupiter.api.Assertions.assertTrue;
     
    @@ -30,46 +32,54 @@
      * cluster creation and cleanup because the command is expected to fail immediately.
      */
     public class LeaderElectionCommandErrorTest {
    +
         @Test
         public void testTopicWithoutPartition() {
    -        String out = ToolsTestUtils.captureStandardErr(() -> LeaderElectionCommand.main(
    -                "--bootstrap-server", "nohost:9092",
    -                "--election-type", "unclean",
    -                "--topic", "some-topic"
    -            ));
    +        String[] args = {
    +            "--bootstrap-server", "nohost:9092",
    +            "--election-type", "unclean",
    +            "--topic", "some-topic"
    +        };
    +        assertEquals(1, LeaderElectionCommand.mainNoExit(args));
    +        String out = ToolsTestUtils.captureStandardErr(() -> LeaderElectionCommand.mainNoExit(args));
             assertTrue(out.startsWith("Missing required option(s)"));
             assertTrue(out.contains(" partition"));
         }
     
         @Test
         public void testPartitionWithoutTopic() {
    -        String out = ToolsTestUtils.captureStandardErr(() -> LeaderElectionCommand.main(
    +        String[] args = {
                 "--bootstrap-server", "nohost:9092",
                 "--election-type", "unclean",
                 "--all-topic-partitions",
                 "--partition", "0"
    -        ));
    -        String[] rows = out.split("\n");
    +        };
    +        assertEquals(1, LeaderElectionCommand.mainNoExit(args));
    +        String out = ToolsTestUtils.captureStandardErr(() -> LeaderElectionCommand.mainNoExit(args));
             assertTrue(out.startsWith("Option partition is only allowed if topic is used"));
         }
     
         @Test
         public void testMissingElectionType() {
    -        String out = ToolsTestUtils.captureStandardErr(() -> LeaderElectionCommand.main(
    +        String[] args = {
                 "--bootstrap-server", "nohost:9092",
                 "--topic", "some-topic",
                 "--partition", "0"
    -        ));
    +        };
    +        assertEquals(1, LeaderElectionCommand.mainNoExit(args));
    +        String out = ToolsTestUtils.captureStandardErr(() -> LeaderElectionCommand.mainNoExit(args));
             assertTrue(out.startsWith("Missing required option(s)"));
             assertTrue(out.contains(" election-type"));
         }
     
         @Test
         public void testMissingTopicPartitionSelection() {
    -        String out = ToolsTestUtils.captureStandardErr(() -> LeaderElectionCommand.main(
    +        String[] args = {
                 "--bootstrap-server", "nohost:9092",
                 "--election-type", "preferred"
    -        ));
    +        };
    +        assertEquals(1, LeaderElectionCommand.mainNoExit(args));
    +        String out = ToolsTestUtils.captureStandardErr(() -> LeaderElectionCommand.mainNoExit(args));
             assertTrue(out.startsWith("One and only one of the following options is required: "));
             assertTrue(out.contains(" all-topic-partitions"));
             assertTrue(out.contains(" topic"));
    @@ -84,6 +94,6 @@ public void testInvalidBroker() {
                 "--election-type", "unclean",
                 "--all-topic-partitions"
             ));
    -        assertTrue(e.getCause() instanceof TimeoutException);
    +        assertInstanceOf(TimeoutException.class, e.getCause());
         }
     }
    diff --git a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java
    index f68b55ed3f..210741b850 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java
    @@ -53,6 +53,7 @@
     import java.util.concurrent.ExecutionException;
     
     import static org.junit.jupiter.api.Assertions.assertEquals;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     import static org.junit.jupiter.api.Assertions.assertThrows;
     import static org.junit.jupiter.api.Assertions.assertTrue;
     
    @@ -104,11 +105,11 @@ public void testAllTopicPartition() throws InterruptedException, ExecutionExcept
             cluster.startBroker(broker3);
             TestUtils.waitForOnlineBroker(client, broker3);
     
    -        LeaderElectionCommand.main(
    +        assertEquals(0, LeaderElectionCommand.mainNoExit(
                 "--bootstrap-server", cluster.bootstrapServers(),
                 "--election-type", "unclean",
                 "--all-topic-partitions"
    -        );
    +        ));
     
             TestUtils.assertLeader(client, topicPartition, broker3);
         }
    @@ -120,11 +121,11 @@ public void testAdminConfigCustomTimeouts() throws Exception {
             Path adminConfigPath = tempAdminConfig(defaultApiTimeoutMs, requestTimeoutMs);
     
             try (final MockedStatic mockedAdmin = Mockito.mockStatic(Admin.class)) {
    -            LeaderElectionCommand.main(
    +            assertEquals(1, LeaderElectionCommand.mainNoExit(
                     "--bootstrap-server", cluster.bootstrapServers(),
                     "--election-type", "unclean", "--all-topic-partitions",
                     "--admin.config", adminConfigPath.toString()
    -            );
    +            ));
     
                 ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(Properties.class);
                 mockedAdmin.verify(() -> Admin.create(argumentCaptor.capture()));
    @@ -160,12 +161,12 @@ public void testTopicPartition() throws InterruptedException, ExecutionException
             cluster.startBroker(broker3);
             TestUtils.waitForOnlineBroker(client, broker3);
     
    -        LeaderElectionCommand.main(
    +        assertEquals(0, LeaderElectionCommand.mainNoExit(
                 "--bootstrap-server", cluster.bootstrapServers(),
                 "--election-type", "unclean",
                 "--topic", topic,
                 "--partition", Integer.toString(partition)
    -        );
    +        ));
     
             TestUtils.assertLeader(client, topicPartition, broker3);
         }
    @@ -199,11 +200,11 @@ public void testPathToJsonFile() throws Exception {
     
             Path topicPartitionPath = tempTopicPartitionFile(Collections.singletonList(topicPartition));
     
    -        LeaderElectionCommand.main(
    +        assertEquals(0, LeaderElectionCommand.mainNoExit(
                 "--bootstrap-server", cluster.bootstrapServers(),
                 "--election-type", "unclean",
                 "--path-to-json-file", topicPartitionPath.toString()
    -        );
    +        ));
     
             TestUtils.assertLeader(client, topicPartition, broker3);
         }
    @@ -232,11 +233,11 @@ public void testPreferredReplicaElection() throws InterruptedException, Executio
                 JavaConverters.asScalaBuffer(Collections.singletonList(broker2)).toSet()
             );
     
    -        LeaderElectionCommand.main(
    +        assertEquals(0, LeaderElectionCommand.mainNoExit(
                 "--bootstrap-server", cluster.bootstrapServers(),
                 "--election-type", "preferred",
                 "--all-topic-partitions"
    -        );
    +        ));
     
             TestUtils.assertLeader(client, topicPartition, broker2);
         }
    @@ -250,7 +251,7 @@ public void testTopicDoesNotExist() {
                 "--topic", "unknown-topic-name",
                 "--partition", "0"
             ));
    -        assertTrue(e.getSuppressed()[0] instanceof UnknownTopicOrPartitionException);
    +        assertInstanceOf(UnknownTopicOrPartitionException.class, e.getSuppressed()[0]);
         }
     
         @ClusterTest
    @@ -287,7 +288,7 @@ public void testElectionResultOutput() throws Exception {
     
             Path topicPartitionPath = tempTopicPartitionFile(Arrays.asList(topicPartition0, topicPartition1));
             String output = ToolsTestUtils.captureStandardOut(() ->
    -            LeaderElectionCommand.main(
    +            LeaderElectionCommand.mainNoExit(
                     "--bootstrap-server", cluster.bootstrapServers(),
                     "--election-type", "preferred",
                     "--path-to-json-file", topicPartitionPath.toString()
    diff --git a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java
    index 4002d84b48..aa858f468a 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java
    @@ -37,6 +37,7 @@
     
     import static org.junit.jupiter.api.Assertions.assertEquals;
     import static org.junit.jupiter.api.Assertions.assertFalse;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     import static org.junit.jupiter.api.Assertions.assertThrows;
     import static org.junit.jupiter.api.Assertions.assertTrue;
     
    @@ -158,19 +159,15 @@ public void testCommandConfig() throws IOException {
     
         @ClusterTest(clusterType = Type.ZK, brokers = 1)
         public void testDescribeQuorumInZkMode() {
    -        assertTrue(
    -            assertThrows(
    +        assertInstanceOf(UnsupportedVersionException.class, assertThrows(
                     ExecutionException.class,
                     () -> MetadataQuorumCommand.execute("--bootstrap-server", cluster.bootstrapServers(), "describe", "--status")
    -            ).getCause() instanceof UnsupportedVersionException
    -        );
    +        ).getCause());
     
    -        assertTrue(
    -            assertThrows(
    +        assertInstanceOf(UnsupportedVersionException.class, assertThrows(
                     ExecutionException.class,
                     () -> MetadataQuorumCommand.execute("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication")
    -            ).getCause() instanceof UnsupportedVersionException
    -        );
    +        ).getCause());
     
         }
     
    diff --git a/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java b/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java
    index fdc732ea29..d427828f2c 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java
    @@ -16,8 +16,8 @@
      */
     package org.apache.kafka.tools;
     
    -import kafka.utils.TestInfoUtils;
     import kafka.server.DynamicConfig;
    +import kafka.utils.TestInfoUtils;
     import kafka.utils.TestUtils;
     import org.apache.kafka.clients.admin.Admin;
     import org.apache.kafka.clients.admin.AlterConfigOp;
    @@ -31,12 +31,14 @@
     import java.io.File;
     import java.io.IOException;
     import java.io.PrintStream;
    +import java.util.AbstractMap.SimpleImmutableEntry;
     import java.util.ArrayList;
     import java.util.Arrays;
     import java.util.Collection;
     import java.util.HashMap;
     import java.util.List;
     import java.util.Map;
    +import java.util.Map.Entry;
     import java.util.Properties;
     import java.util.Set;
     import java.util.concurrent.ExecutionException;
    @@ -45,6 +47,8 @@
     public class ToolsTestUtils {
         /** @see TestInfoUtils#TestWithParameterizedQuorumName()  */
         public static final String TEST_WITH_PARAMETERIZED_QUORUM_NAME = "{displayName}.{argumentsWithNames}";
    +    /** @see TestInfoUtils#TestWithParameterizedQuorumAndGroupProtocolNames()  */
    +    public static final String TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES = "{displayName}.quorum={0}.groupProtocol={1}";
     
         private static int randomPort = 0;
     
    @@ -153,7 +157,7 @@ public static void removeReplicationThrottleForPartitions(Admin admin, List> allReplicasByPartition) throws InterruptedException, ExecutionException {
    -        Map>>> configResourceToPartitionReplicas =
    +        Map>>> configResourceToPartitionReplicas =
                 allReplicasByPartition.entrySet().stream()
                 .collect(Collectors.groupingBy(
                     topicPartitionListEntry -> new ConfigResource(ConfigResource.Type.TOPIC, topicPartitionListEntry.getKey().topic()))
    @@ -161,10 +165,10 @@ public static void assignThrottledPartitionReplicas(Admin adminClient, Map> throttles = configResourceToPartitionReplicas.entrySet().stream()
                 .collect(
    -                Collectors.toMap(Map.Entry::getKey, entry -> {
    +                Collectors.toMap(Entry::getKey, entry -> {
                         List alterConfigOps = new ArrayList<>();
                         Map> replicaThrottle =
    -                        entry.getValue().stream().collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
    +                        entry.getValue().stream().collect(Collectors.toMap(Entry::getKey, Entry::getValue));
                         alterConfigOps.add(new AlterConfigOp(
                             new ConfigEntry(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, formatReplicaThrottles(replicaThrottle)),
                             AlterConfigOp.OpType.SET));
    @@ -202,12 +206,72 @@ public static String formatReplicaThrottles(Map> m
     
         public static File tempPropertiesFile(Map properties) throws IOException {
             StringBuilder sb = new StringBuilder();
    -        for (Map.Entry entry : properties.entrySet()) {
    +        for (Entry entry : properties.entrySet()) {
                 sb.append(entry.getKey() + "=" + entry.getValue() + System.lineSeparator());
             }
             return org.apache.kafka.test.TestUtils.tempFile(sb.toString());
         }
     
    +    /**
    +     * Capture the console output during the execution of the provided function.
    +     */
    +    public static String grabConsoleOutput(Runnable f) {
    +        ByteArrayOutputStream buf = new ByteArrayOutputStream();
    +        PrintStream out = new PrintStream(buf);
    +        PrintStream out0 = System.out;
    +
    +        System.setOut(out);
    +        try {
    +            f.run();
    +        } finally {
    +            System.setOut(out0);
    +        }
    +        out.flush();
    +        return buf.toString();
    +    }
    +
    +    /**
    +     * Capture the console error during the execution of the provided function.
    +     */
    +    public static String grabConsoleError(Runnable f) {
    +        ByteArrayOutputStream buf = new ByteArrayOutputStream();
    +        PrintStream err = new PrintStream(buf);
    +        PrintStream err0 = System.err;
    +
    +        System.setErr(err);
    +        try {
    +            f.run();
    +        } finally {
    +            System.setErr(err0);
    +        }
    +        err.flush();
    +        return buf.toString();
    +    }
    +
    +    /**
    +     * Capture both the console output and console error during the execution of the provided function.
    +     */
    +    public static Entry grabConsoleOutputAndError(Runnable f) {
    +        ByteArrayOutputStream outBuf = new ByteArrayOutputStream();
    +        ByteArrayOutputStream errBuf = new ByteArrayOutputStream();
    +        PrintStream out = new PrintStream(outBuf);
    +        PrintStream err = new PrintStream(errBuf);
    +        PrintStream out0 = System.out;
    +        PrintStream err0 = System.err;
    +
    +        System.setOut(out);
    +        System.setErr(err);
    +        try {
    +            f.run();
    +        } finally {
    +            System.setOut(out0);
    +            System.setErr(err0);
    +        }
    +        out.flush();
    +        err.flush();
    +        return new SimpleImmutableEntry<>(outBuf.toString(), errBuf.toString());
    +    }
    +
         public static class MockExitProcedure implements Exit.Procedure {
             private boolean hasExited = false;
             private int statusCode;
    diff --git a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java
    index 6831b0bae4..7541d0a112 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java
    @@ -50,6 +50,7 @@
     
     import static org.junit.jupiter.api.Assertions.assertEquals;
     import static org.junit.jupiter.api.Assertions.assertFalse;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     import static org.junit.jupiter.api.Assertions.assertThrows;
     import static org.junit.jupiter.api.Assertions.assertTrue;
     import static org.mockito.ArgumentMatchers.any;
    @@ -233,7 +234,7 @@ public void testDeleteTopicDoesNotRetryThrottlingQuotaExceededException() {
                     "--delete", "--topic", topicName
                 })));
     
    -        assertTrue(exception.getCause() instanceof ThrottlingQuotaExceededException);
    +        assertInstanceOf(ThrottlingQuotaExceededException.class, exception.getCause());
     
             verify(adminClient).deleteTopics(
                 argThat((Collection topics) -> topics.equals(Arrays.asList(topicName))),
    @@ -262,7 +263,7 @@ public void testCreatePartitionsDoesNotRetryThrottlingQuotaExceededException() {
                     "--alter", "--topic", topicName, "--partitions", "3",
                     "--bootstrap-server", bootstrapServer
                 })));
    -        assertTrue(exception.getCause() instanceof ThrottlingQuotaExceededException);
    +        assertInstanceOf(ThrottlingQuotaExceededException.class, exception.getCause());
     
             verify(adminClient, times(1)).createPartitions(
                 argThat(newPartitions -> newPartitions.get(topicName).totalCount() == 3),
    diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java
    index 523122c4cd..c7d3a82232 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java
    @@ -32,6 +32,7 @@
     
     import static org.junit.jupiter.api.Assertions.assertEquals;
     import static org.junit.jupiter.api.Assertions.assertFalse;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
     import static org.junit.jupiter.api.Assertions.assertThrows;
     import static org.junit.jupiter.api.Assertions.assertTrue;
     
    @@ -48,12 +49,12 @@ public void shouldParseValidConsumerValidConfig() throws IOException {
             ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
     
             assertEquals("localhost:9092", config.bootstrapServer());
    -        assertEquals("test", config.topicArg());
    +        assertEquals("test", config.topicArg().orElse(""));
             assertTrue(config.fromBeginning());
             assertFalse(config.enableSystestEventsLogging());
             assertFalse(config.skipMessageOnError());
             assertEquals(-1, config.maxMessages());
    -        assertEquals(-1, config.timeoutMs());
    +        assertEquals(Long.MAX_VALUE, config.timeoutMs());
         }
     
         @Test
    @@ -67,7 +68,7 @@ public void shouldParseIncludeArgument() throws IOException {
             ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
     
             assertEquals("localhost:9092", config.bootstrapServer());
    -        assertEquals("includeTest*", config.includedTopicsArg());
    +        assertEquals("includeTest*", config.includedTopicsArg().orElse(""));
             assertTrue(config.fromBeginning());
         }
     
    @@ -82,7 +83,7 @@ public void shouldParseWhitelistArgument() throws IOException {
             ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
     
             assertEquals("localhost:9092", config.bootstrapServer());
    -        assertEquals("whitelistTest*", config.includedTopicsArg());
    +        assertEquals("whitelistTest*", config.includedTopicsArg().orElse(""));
             assertTrue(config.fromBeginning());
         }
     
    @@ -96,7 +97,7 @@ public void shouldIgnoreWhitelistArgumentIfIncludeSpecified() throws IOException
             };
             ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
             assertEquals("localhost:9092", config.bootstrapServer());
    -        assertEquals("includeTest*", config.includedTopicsArg());
    +        assertEquals("includeTest*", config.includedTopicsArg().orElse(""));
             assertTrue(config.fromBeginning());
         }
     
    @@ -112,7 +113,7 @@ public void shouldParseValidSimpleConsumerValidConfigWithNumericOffset() throws
             ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
     
             assertEquals("localhost:9092", config.bootstrapServer());
    -        assertEquals("test", config.topicArg());
    +        assertEquals("test", config.topicArg().orElse(""));
             assertTrue(config.partitionArg().isPresent());
             assertEquals(0, config.partitionArg().getAsInt());
             assertEquals(3, config.offsetArg());
    @@ -191,7 +192,7 @@ public void shouldParseValidSimpleConsumerValidConfigWithStringOffset() throws E
             ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
     
             assertEquals("localhost:9092", config.bootstrapServer());
    -        assertEquals("test", config.topicArg());
    +        assertEquals("test", config.topicArg().orElse(""));
             assertTrue(config.partitionArg().isPresent());
             assertEquals(0, config.partitionArg().getAsInt());
             assertEquals(-1, config.offsetArg());
    @@ -211,7 +212,7 @@ public void shouldParseValidConsumerConfigWithAutoOffsetResetLatest() throws IOE
             Properties consumerProperties = config.consumerProps();
     
             assertEquals("localhost:9092", config.bootstrapServer());
    -        assertEquals("test", config.topicArg());
    +        assertEquals("test", config.topicArg().orElse(""));
             assertFalse(config.fromBeginning());
             assertEquals("latest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG));
         }
    @@ -228,7 +229,7 @@ public void shouldParseValidConsumerConfigWithAutoOffsetResetEarliest() throws I
             Properties consumerProperties = config.consumerProps();
     
             assertEquals("localhost:9092", config.bootstrapServer());
    -        assertEquals("test", config.topicArg());
    +        assertEquals("test", config.topicArg().orElse(""));
             assertFalse(config.fromBeginning());
             assertEquals("earliest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG));
         }
    @@ -246,7 +247,7 @@ public void shouldParseValidConsumerConfigWithAutoOffsetResetAndMatchingFromBegi
             Properties consumerProperties = config.consumerProps();
     
             assertEquals("localhost:9092", config.bootstrapServer());
    -        assertEquals("test", config.topicArg());
    +        assertEquals("test", config.topicArg().orElse(""));
             assertTrue(config.fromBeginning());
             assertEquals("earliest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG));
         }
    @@ -262,7 +263,7 @@ public void shouldParseValidConsumerConfigWithNoOffsetReset() throws IOException
             Properties consumerProperties = config.consumerProps();
     
             assertEquals("localhost:9092", config.bootstrapServer());
    -        assertEquals("test", config.topicArg());
    +        assertEquals("test", config.topicArg().orElse(""));
             assertFalse(config.fromBeginning());
             assertEquals("latest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG));
         }
    @@ -392,11 +393,11 @@ public void testCustomPropertyShouldBePassedToConfigureMethod() throws Exception
     
             ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
     
    -        assertTrue(config.formatter() instanceof DefaultMessageFormatter);
    +        assertInstanceOf(DefaultMessageFormatter.class, config.formatter());
             assertTrue(config.formatterArgs().containsKey("key.deserializer.my-props"));
             DefaultMessageFormatter formatter = (DefaultMessageFormatter) config.formatter();
             assertTrue(formatter.keyDeserializer().isPresent());
    -        assertTrue(formatter.keyDeserializer().get() instanceof MockDeserializer);
    +        assertInstanceOf(MockDeserializer.class, formatter.keyDeserializer().get());
             MockDeserializer keyDeserializer = (MockDeserializer) formatter.keyDeserializer().get();
             assertEquals(1, keyDeserializer.configs.size());
             assertEquals("abc", keyDeserializer.configs.get("my-props"));
    @@ -419,11 +420,11 @@ public void testCustomConfigShouldBePassedToConfigureMethod() throws Exception {
     
             ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
     
    -        assertTrue(config.formatter() instanceof DefaultMessageFormatter);
    +        assertInstanceOf(DefaultMessageFormatter.class, config.formatter());
             assertTrue(config.formatterArgs().containsKey("key.deserializer.my-props"));
             DefaultMessageFormatter formatter = (DefaultMessageFormatter) config.formatter();
             assertTrue(formatter.keyDeserializer().isPresent());
    -        assertTrue(formatter.keyDeserializer().get() instanceof MockDeserializer);
    +        assertInstanceOf(MockDeserializer.class, formatter.keyDeserializer().get());
             MockDeserializer keyDeserializer = (MockDeserializer) formatter.keyDeserializer().get();
             assertEquals(1, keyDeserializer.configs.size());
             assertEquals("abc", keyDeserializer.configs.get("my-props"));
    @@ -442,7 +443,7 @@ public void shouldParseGroupIdFromBeginningGivenTogether() throws IOException {
     
             ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
             assertEquals("localhost:9092", config.bootstrapServer());
    -        assertEquals("test", config.topicArg());
    +        assertEquals("test", config.topicArg().orElse(""));
             assertEquals(-2, config.offsetArg());
             assertTrue(config.fromBeginning());
     
    @@ -455,7 +456,7 @@ public void shouldParseGroupIdFromBeginningGivenTogether() throws IOException {
     
             config = new ConsoleConsumerOptions(args);
             assertEquals("localhost:9092", config.bootstrapServer());
    -        assertEquals("test", config.topicArg());
    +        assertEquals("test", config.topicArg().orElse(""));
             assertEquals(-1, config.offsetArg());
             assertFalse(config.fromBeginning());
         }
    @@ -618,4 +619,30 @@ public void testParseOffset() throws Exception {
                 Exit.resetExitProcedure();
             }
         }
    +
    +    @Test
    +    public void testParseTimeoutMs() throws Exception {
    +        String[] withoutTimeoutMs = new String[]{
    +            "--bootstrap-server", "localhost:9092",
    +            "--topic", "test",
    +            "--partition", "0"
    +        };
    +        assertEquals(Long.MAX_VALUE, new ConsoleConsumerOptions(withoutTimeoutMs).timeoutMs());
    +
    +        String[] negativeTimeoutMs = new String[]{
    +            "--bootstrap-server", "localhost:9092",
    +            "--topic", "test",
    +            "--partition", "0",
    +            "--timeout-ms", "-100"
    +        };
    +        assertEquals(Long.MAX_VALUE, new ConsoleConsumerOptions(negativeTimeoutMs).timeoutMs());
    +
    +        String[] validTimeoutMs = new String[]{
    +            "--bootstrap-server", "localhost:9092",
    +            "--topic", "test",
    +            "--partition", "0",
    +            "--timeout-ms", "100"
    +        };
    +        assertEquals(100, new ConsoleConsumerOptions(validTimeoutMs).timeoutMs());
    +    }
     }
    diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java
    index 008893f9c5..f4fa6ac3be 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java
    @@ -24,21 +24,19 @@
     import org.apache.kafka.common.MessageFormatter;
     import org.apache.kafka.common.TopicPartition;
     import org.apache.kafka.common.errors.TimeoutException;
    -import org.apache.kafka.common.requests.ListOffsetsRequest;
     import org.apache.kafka.common.utils.Time;
     import org.apache.kafka.server.util.MockTime;
     import org.junit.jupiter.api.BeforeEach;
     import org.junit.jupiter.api.Test;
     
    +import java.io.IOException;
     import java.io.PrintStream;
     import java.time.Duration;
     import java.util.Arrays;
     import java.util.Collections;
     import java.util.HashMap;
     import java.util.Map;
    -import java.util.Optional;
    -import java.util.OptionalInt;
    -import java.util.OptionalLong;
    +import java.util.regex.Pattern;
     
     import static org.junit.jupiter.api.Assertions.assertEquals;
     import static org.junit.jupiter.api.Assertions.assertThrows;
    @@ -58,8 +56,7 @@ public void setup() {
         }
     
         @Test
    -    public void shouldThrowTimeoutExceptionWhenTimeoutIsReached() {
    -        String topic = "test";
    +    public void shouldThrowTimeoutExceptionWhenTimeoutIsReached() throws IOException {
             final Time time = new MockTime();
             final int timeoutMs = 1000;
     
    @@ -71,20 +68,22 @@ public void shouldThrowTimeoutExceptionWhenTimeoutIsReached() {
                 return ConsumerRecords.EMPTY;
             });
     
    +        String[] args = new String[]{
    +            "--bootstrap-server", "localhost:9092",
    +            "--topic", "test",
    +            "--timeout-ms", String.valueOf(timeoutMs)
    +        };
    +
             ConsoleConsumer.ConsumerWrapper consumer = new ConsoleConsumer.ConsumerWrapper(
    -                Optional.of(topic),
    -                OptionalInt.empty(),
    -                OptionalLong.empty(),
    -                Optional.empty(),
    -                mockConsumer,
    -                timeoutMs
    +            new ConsoleConsumerOptions(args),
    +            mockConsumer
             );
     
             assertThrows(TimeoutException.class, consumer::receive);
         }
     
         @Test
    -    public void shouldResetUnConsumedOffsetsBeforeExit() {
    +    public void shouldResetUnConsumedOffsetsBeforeExit() throws IOException {
             String topic = "test";
             int maxMessages = 123;
             int totalMessages = 700;
    @@ -94,13 +93,16 @@ public void shouldResetUnConsumedOffsetsBeforeExit() {
             TopicPartition tp1 = new TopicPartition(topic, 0);
             TopicPartition tp2 = new TopicPartition(topic, 1);
     
    +        String[] args = new String[]{
    +            "--bootstrap-server", "localhost:9092",
    +            "--topic", topic,
    +            "--timeout-ms", "1000"
    +        };
    +
             ConsoleConsumer.ConsumerWrapper consumer = new ConsoleConsumer.ConsumerWrapper(
    -                Optional.of(topic),
    -                OptionalInt.empty(),
    -                OptionalLong.empty(),
    -                Optional.empty(),
    -                mockConsumer,
    -                1000L);
    +            new ConsoleConsumerOptions(args),
    +            mockConsumer
    +        );
     
             mockConsumer.rebalance(Arrays.asList(tp1, tp2));
             Map offsets = new HashMap<>();
    @@ -165,47 +167,75 @@ public void shouldStopWhenOutputCheckErrorFails() {
     
         @Test
         @SuppressWarnings("unchecked")
    -    public void shouldSeekWhenOffsetIsSet() {
    +    public void shouldSeekWhenOffsetIsSet() throws IOException {
             Consumer mockConsumer = mock(Consumer.class);
             TopicPartition tp0 = new TopicPartition("test", 0);
     
    +        String[] args = new String[]{
    +            "--bootstrap-server", "localhost:9092",
    +            "--topic", tp0.topic(),
    +            "--partition", String.valueOf(tp0.partition()),
    +            "--timeout-ms", "1000"
    +        };
    +
             ConsoleConsumer.ConsumerWrapper consumer = new ConsoleConsumer.ConsumerWrapper(
    -                Optional.of(tp0.topic()),
    -                OptionalInt.of(tp0.partition()),
    -                OptionalLong.empty(),
    -                Optional.empty(),
    -                mockConsumer,
    -                1000L);
    +            new ConsoleConsumerOptions(args),
    +            mockConsumer
    +        );
     
             verify(mockConsumer).assign(eq(Collections.singletonList(tp0)));
             verify(mockConsumer).seekToEnd(eq(Collections.singletonList(tp0)));
             consumer.cleanup();
             reset(mockConsumer);
     
    -        consumer = new ConsoleConsumer.ConsumerWrapper(
    -                Optional.of(tp0.topic()),
    -                OptionalInt.of(tp0.partition()),
    -                OptionalLong.of(123L),
    -                Optional.empty(),
    -                mockConsumer,
    -                1000L);
    +        args = new String[]{
    +            "--bootstrap-server", "localhost:9092",
    +            "--topic", tp0.topic(),
    +            "--partition", String.valueOf(tp0.partition()),
    +            "--offset", "123",
    +            "--timeout-ms", "1000"
    +        };
    +
    +        consumer = new ConsoleConsumer.ConsumerWrapper(new ConsoleConsumerOptions(args), mockConsumer);
     
             verify(mockConsumer).assign(eq(Collections.singletonList(tp0)));
             verify(mockConsumer).seek(eq(tp0), eq(123L));
             consumer.cleanup();
             reset(mockConsumer);
     
    -        consumer = new ConsoleConsumer.ConsumerWrapper(
    -                Optional.of(tp0.topic()),
    -                OptionalInt.of(tp0.partition()),
    -                OptionalLong.of(ListOffsetsRequest.EARLIEST_TIMESTAMP),
    -                Optional.empty(),
    -                mockConsumer,
    -                1000L);
    +        args = new String[]{
    +            "--bootstrap-server", "localhost:9092",
    +            "--topic", tp0.topic(),
    +            "--partition", String.valueOf(tp0.partition()),
    +            "--offset", "earliest",
    +            "--timeout-ms", "1000"
    +        };
    +
    +        consumer = new ConsoleConsumer.ConsumerWrapper(new ConsoleConsumerOptions(args), mockConsumer);
     
             verify(mockConsumer).assign(eq(Collections.singletonList(tp0)));
             verify(mockConsumer).seekToBeginning(eq(Collections.singletonList(tp0)));
             consumer.cleanup();
             reset(mockConsumer);
         }
    +
    +    @Test
    +    @SuppressWarnings("unchecked")
    +    public void shouldWorkWithoutTopicOption() throws IOException {
    +        Consumer mockConsumer = mock(Consumer.class);
    +
    +        String[] args = new String[]{
    +            "--bootstrap-server", "localhost:9092",
    +            "--include", "includeTest*",
    +            "--from-beginning"
    +        };
    +
    +        ConsoleConsumer.ConsumerWrapper consumer = new ConsoleConsumer.ConsumerWrapper(
    +            new ConsoleConsumerOptions(args),
    +            mockConsumer
    +        );
    +
    +        verify(mockConsumer).subscribe(any(Pattern.class));
    +        consumer.cleanup();
    +    }
     }
    diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java
    new file mode 100644
    index 0000000000..a0ad01bad9
    --- /dev/null
    +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.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.tools.consumer.group;
    +
    +import kafka.api.AbstractAuthorizerIntegrationTest;
    +import kafka.security.authorizer.AclEntry;
    +import org.apache.kafka.common.acl.AccessControlEntry;
    +import org.junit.jupiter.params.ParameterizedTest;
    +import org.junit.jupiter.params.provider.ValueSource;
    +import scala.collection.JavaConverters;
    +
    +import java.util.Collections;
    +
    +import static org.apache.kafka.common.acl.AclOperation.DESCRIBE;
    +import static org.apache.kafka.common.acl.AclPermissionType.ALLOW;
    +import static org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME;
    +
    +public class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
    +    @SuppressWarnings({"deprecation"})
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
    +    @ValueSource(strings = {"zk", "kraft"})
    +    public void testDescribeGroupCliWithGroupDescribe(String quorum) throws Exception {
    +        addAndVerifyAcls(JavaConverters.asScalaSet(Collections.singleton(new AccessControlEntry(ClientPrincipal().toString(), AclEntry.WildcardHost(), DESCRIBE, ALLOW))).toSet(), groupResource());
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group()};
    +        ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(cgcArgs);
    +        ConsumerGroupCommand.ConsumerGroupService consumerGroupService = new ConsumerGroupCommand.ConsumerGroupService(opts, Collections.emptyMap());
    +        consumerGroupService.describeGroups();
    +        consumerGroupService.close();
    +    }
    +}
    diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java
    index b78054cb4a..e1daa37f19 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java
    @@ -17,7 +17,6 @@
     package org.apache.kafka.tools.consumer.group;
     
     import kafka.api.BaseConsumerTest;
    -import kafka.admin.ConsumerGroupCommand;
     import kafka.server.KafkaConfig;
     import kafka.utils.TestUtils;
     import org.apache.kafka.clients.admin.AdminClientConfig;
    @@ -41,7 +40,6 @@
     import java.util.Arrays;
     import java.util.Collection;
     import java.util.Collections;
    -import java.util.HashSet;
     import java.util.List;
     import java.util.Locale;
     import java.util.Map;
    @@ -58,6 +56,7 @@
     public class ConsumerGroupCommandTest extends kafka.integration.KafkaServerTestHarness {
         public static final String TOPIC = "foo";
         public static final String GROUP = "test.group";
    +    public static final String PROTOCOL_GROUP = "protocol-group";
     
         List consumerGroupService = new ArrayList<>();
         List consumerGroupExecutors = new ArrayList<>();
    @@ -86,10 +85,7 @@ public Seq generateConfigs() {
                 0,
                 false
             ).foreach(props -> {
    -            if (isNewGroupCoordinatorEnabled()) {
    -                props.setProperty(KafkaConfig.NewGroupCoordinatorEnableProp(), "true");
    -            }
    -
    +            props.setProperty(KafkaConfig.NewGroupCoordinatorEnableProp(), isNewGroupCoordinatorEnabled() + "");
                 cfgs.add(KafkaConfig.fromProps(props));
                 return null;
             });
    @@ -132,10 +128,10 @@ Consumer createNoAutoCommitConsumer(String group) {
         }
     
         ConsumerGroupCommand.ConsumerGroupService getConsumerGroupService(String[] args) {
    -        ConsumerGroupCommand.ConsumerGroupCommandOptions opts = new ConsumerGroupCommand.ConsumerGroupCommandOptions(args);
    +        ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(args);
             ConsumerGroupCommand.ConsumerGroupService service = new ConsumerGroupCommand.ConsumerGroupService(
                 opts,
    -            asScala(Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE)))
    +            Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE))
             );
     
             consumerGroupService.add(0, service);
    @@ -154,8 +150,8 @@ ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers, String groupPro
             return addConsumerGroupExecutor(numConsumers, TOPIC, GROUP, RangeAssignor.class.getName(), remoteAssignor, Optional.empty(), false, groupProtocol);
         }
     
    -    ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers, String topic, String group) {
    -        return addConsumerGroupExecutor(numConsumers, topic, group, RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), false, GroupProtocol.CLASSIC.name);
    +    ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers, String group, String groupProtocol) {
    +        return addConsumerGroupExecutor(numConsumers, TOPIC, group, RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), false, groupProtocol);
         }
     
         ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers, String topic, String group, String groupProtocol) {
    @@ -342,18 +338,16 @@ public static Stream getTestQuorumAndGroupProtocolParametersAll() {
             return BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll();
         }
     
    -    @SuppressWarnings({"deprecation"})
    -    static  Seq seq(Collection seq) {
    -        return JavaConverters.asScalaIteratorConverter(seq.iterator()).asScala().toSeq();
    +    public static Stream getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly() {
    +        return BaseConsumerTest.getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly();
         }
     
    -    @SuppressWarnings("deprecation")
    -    static  scala.collection.Map asScala(Map jmap) {
    -        return JavaConverters.mapAsScalaMap(jmap);
    +    public static Stream getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly() {
    +        return BaseConsumerTest.getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly();
         }
     
         @SuppressWarnings({"deprecation"})
    -    static  scala.collection.immutable.Set set(final Collection set) {
    -        return JavaConverters.asScalaSet(new HashSet<>(set)).toSet();
    +    static  Seq seq(Collection seq) {
    +        return JavaConverters.asScalaIteratorConverter(seq.iterator()).asScala().toSeq();
         }
     }
    diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupServiceTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupServiceTest.java
    index 8c75823648..4fd7e3b919 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupServiceTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupServiceTest.java
    @@ -16,7 +16,6 @@
      */
     package org.apache.kafka.tools.consumer.group;
     
    -import kafka.admin.ConsumerGroupCommand;
     import org.apache.kafka.clients.admin.Admin;
     import org.apache.kafka.clients.admin.AdminClientTestUtils;
     import org.apache.kafka.clients.admin.ConsumerGroupDescription;
    @@ -42,12 +41,6 @@
     import org.junit.jupiter.api.Test;
     import org.mockito.ArgumentMatcher;
     import org.mockito.ArgumentMatchers;
    -import scala.Option;
    -import scala.Some;
    -import scala.Tuple2;
    -import scala.collection.JavaConverters;
    -import scala.collection.Seq;
    -import scala.collection.immutable.Map$;
     
     import java.util.ArrayList;
     import java.util.Arrays;
    @@ -55,11 +48,12 @@
     import java.util.Collections;
     import java.util.HashMap;
     import java.util.HashSet;
    -import java.util.Objects;
    -import java.util.Set;
     import java.util.List;
     import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Objects;
     import java.util.Optional;
    +import java.util.Set;
     import java.util.function.Function;
     import java.util.stream.Collectors;
     import java.util.stream.IntStream;
    @@ -86,7 +80,7 @@ public class ConsumerGroupServiceTest {
         private final Admin admin = mock(Admin.class);
     
         @Test
    -    public void testAdminRequestsForDescribeOffsets() {
    +    public void testAdminRequestsForDescribeOffsets() throws Exception {
             String[] args = new String[]{"--bootstrap-server", "localhost:9092", "--group", GROUP, "--describe", "--offsets"};
             ConsumerGroupCommand.ConsumerGroupService groupService = consumerGroupService(args);
     
    @@ -97,10 +91,10 @@ public void testAdminRequestsForDescribeOffsets() {
             when(admin.listOffsets(offsetsArgMatcher(), any()))
                     .thenReturn(listOffsetsResult());
     
    -        Tuple2, Option>> statesAndAssignments = groupService.collectGroupOffsets(GROUP);
    -        assertEquals(Some.apply("Stable"), statesAndAssignments._1);
    -        assertTrue(statesAndAssignments._2.isDefined());
    -        assertEquals(TOPIC_PARTITIONS.size(), statesAndAssignments._2.get().size());
    +        Entry, Optional>> statesAndAssignments = groupService.collectGroupOffsets(GROUP);
    +        assertEquals(Optional.of("Stable"), statesAndAssignments.getKey());
    +        assertTrue(statesAndAssignments.getValue().isPresent());
    +        assertEquals(TOPIC_PARTITIONS.size(), statesAndAssignments.getValue().get().size());
     
             verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(GROUP)), any());
             verify(admin, times(1)).listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec()), any());
    @@ -108,7 +102,7 @@ public void testAdminRequestsForDescribeOffsets() {
         }
     
         @Test
    -    public void testAdminRequestsForDescribeNegativeOffsets() {
    +    public void testAdminRequestsForDescribeNegativeOffsets() throws Exception {
             String[] args = new String[]{"--bootstrap-server", "localhost:9092", "--group", GROUP, "--describe", "--offsets"};
             ConsumerGroupCommand.ConsumerGroupService groupService = consumerGroupService(args);
     
    @@ -163,27 +157,22 @@ public void testAdminRequestsForDescribeNegativeOffsets() {
                     ArgumentMatchers.argThat(offsetsArgMatcher.apply(assignedTopicPartitions)),
                     any()
             )).thenReturn(new ListOffsetsResult(endOffsets.entrySet().stream().filter(e -> assignedTopicPartitions.contains(e.getKey()))
    -                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))));
    +                .collect(Collectors.toMap(Entry::getKey, Entry::getValue))));
             when(admin.listOffsets(
                     ArgumentMatchers.argThat(offsetsArgMatcher.apply(unassignedTopicPartitions)),
                     any()
             )).thenReturn(new ListOffsetsResult(endOffsets.entrySet().stream().filter(e -> unassignedTopicPartitions.contains(e.getKey()))
    -                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))));
    -
    -        Tuple2, Option>> statesAndAssignments = groupService.collectGroupOffsets(GROUP);
    -        Option state = statesAndAssignments._1;
    -        Option> assignments = statesAndAssignments._2;
    -
    -        Map> returnedOffsets = new HashMap<>();
    -        assignments.foreach(results -> {
    -            results.foreach(assignment -> {
    -                returnedOffsets.put(
    -                        new TopicPartition(assignment.topic().get(), (Integer) assignment.partition().get()),
    -                        assignment.offset().isDefined() ? Optional.of((Long) assignment.offset().get()) : Optional.empty());
    -                return null;
    -            });
    -            return null;
    -        });
    +                .collect(Collectors.toMap(Entry::getKey, Entry::getValue))));
    +
    +        Entry, Optional>> statesAndAssignments = groupService.collectGroupOffsets(GROUP);
    +        Optional state = statesAndAssignments.getKey();
    +        Optional> assignments = statesAndAssignments.getValue();
    +
    +        Map> returnedOffsets = assignments.map(results ->
    +            results.stream().collect(Collectors.toMap(
    +                assignment -> new TopicPartition(assignment.topic.get(), assignment.partition.get()),
    +                assignment -> assignment.offset))
    +        ).orElse(Collections.emptyMap());
     
             Map> expectedOffsets = new HashMap<>();
     
    @@ -194,7 +183,7 @@ public void testAdminRequestsForDescribeNegativeOffsets() {
             expectedOffsets.put(testTopicPartition4, Optional.of(100L));
             expectedOffsets.put(testTopicPartition5, Optional.empty());
     
    -        assertEquals(Some.apply("Stable"), state);
    +        assertEquals(Optional.of("Stable"), state);
             assertEquals(expectedOffsets, returnedOffsets);
     
             verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(GROUP)), any());
    @@ -220,9 +209,9 @@ public void testAdminRequestsForResetOffsets() {
             when(admin.listOffsets(offsetsArgMatcher(), any()))
                     .thenReturn(listOffsetsResult());
     
    -        scala.collection.Map> resetResult = groupService.resetOffsets();
    -        assertEquals(set(Collections.singletonList(GROUP)), resetResult.keySet());
    -        assertEquals(set(TOPIC_PARTITIONS), resetResult.get(GROUP).get().keys().toSet());
    +        Map> resetResult = groupService.resetOffsets();
    +        assertEquals(Collections.singleton(GROUP), resetResult.keySet());
    +        assertEquals(new HashSet<>(TOPIC_PARTITIONS), resetResult.get(GROUP).keySet());
     
             verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(GROUP)), any());
             verify(admin, times(1)).describeTopics(ArgumentMatchers.eq(topicsWithoutPartitionsSpecified), any());
    @@ -230,9 +219,9 @@ public void testAdminRequestsForResetOffsets() {
         }
     
         private ConsumerGroupCommand.ConsumerGroupService consumerGroupService(String[] args) {
    -        return new ConsumerGroupCommand.ConsumerGroupService(new kafka.admin.ConsumerGroupCommand.ConsumerGroupCommandOptions(args), Map$.MODULE$.empty()) {
    +        return new ConsumerGroupCommand.ConsumerGroupService(ConsumerGroupCommandOptions.fromArgs(args), Collections.emptyMap()) {
                 @Override
    -            public Admin createAdminClient(scala.collection.Map configOverrides) {
    +            protected Admin createAdminClient(Map configOverrides) {
                     return admin;
                 }
             };
    @@ -291,9 +280,4 @@ private DescribeTopicsResult describeTopicsResult(Collection topics) {
         private Map listConsumerGroupOffsetsSpec() {
             return Collections.singletonMap(GROUP, new ListConsumerGroupOffsetsSpec());
         }
    -
    -    @SuppressWarnings({"deprecation"})
    -    private static  scala.collection.immutable.Set set(final Collection set) {
    -        return JavaConverters.asScalaSet(new HashSet<>(set)).toSet();
    -    }
     }
    diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java
    index 0a1bcb2dae..b8133433d6 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java
    @@ -17,17 +17,18 @@
     package org.apache.kafka.tools.consumer.group;
     
     import joptsimple.OptionException;
    -import kafka.admin.ConsumerGroupCommand;
     import org.apache.kafka.clients.consumer.GroupProtocol;
     import org.apache.kafka.clients.consumer.RangeAssignor;
     import org.apache.kafka.common.errors.GroupIdNotFoundException;
     import org.apache.kafka.common.errors.GroupNotEmptyException;
     import org.apache.kafka.common.protocol.Errors;
     import org.apache.kafka.test.TestUtils;
    +import org.apache.kafka.tools.ToolsTestUtils;
     import org.junit.jupiter.params.ParameterizedTest;
     import org.junit.jupiter.params.provider.ValueSource;
     
     import java.util.Arrays;
    +import java.util.HashSet;
     import java.util.Map;
     import java.util.Objects;
     import java.util.Optional;
    @@ -60,10 +61,7 @@ public void testDeleteCmdNonExistingGroup(String quorum) {
             String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
             ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
     
    -        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
    -            service.deleteGroups();
    -            return null;
    -        });
    +        String output = ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
             assertTrue(output.contains("Group '" + missingGroup + "' could not be deleted due to:") && output.contains(Errors.GROUP_ID_NOT_FOUND.message()),
                 "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not detected while deleting consumer group");
         }
    @@ -78,8 +76,8 @@ public void testDeleteNonExistingGroup(String quorum) {
             String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
             ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
     
    -        scala.collection.Map result = service.deleteGroups();
    -        assertTrue(result.size() == 1 && result.contains(missingGroup) && result.get(missingGroup).get().getCause() instanceof GroupIdNotFoundException,
    +        Map result = service.deleteGroups();
    +        assertTrue(result.size() == 1 && result.containsKey(missingGroup) && result.get(missingGroup).getCause() instanceof GroupIdNotFoundException,
                 "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not detected while deleting consumer group");
         }
     
    @@ -94,14 +92,11 @@ public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
             ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
     
             TestUtils.waitForCondition(
    -            () -> service.collectGroupMembers(GROUP, false)._2.get().size() == 1,
    +            () -> service.collectGroupMembers(GROUP, false).getValue().get().size() == 1,
                 "The group did not initialize as expected."
             );
     
    -        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
    -            service.deleteGroups();
    -            return null;
    -        });
    +        String output = ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
             assertTrue(output.contains("Group '" + GROUP + "' could not be deleted due to:") && output.contains(Errors.NON_EMPTY_GROUP.message()),
                 "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not detected while deleting consumer group. Output was: (" + output + ")");
         }
    @@ -117,14 +112,14 @@ public void testDeleteNonEmptyGroup(String quorum) throws Exception {
             ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
     
             TestUtils.waitForCondition(
    -            () -> service.collectGroupMembers(GROUP, false)._2.get().size() == 1,
    +            () -> service.collectGroupMembers(GROUP, false).getValue().get().size() == 1,
                 "The group did not initialize as expected."
             );
     
    -        scala.collection.Map result = service.deleteGroups();
    -        assertNotNull(result.get(GROUP).get(),
    +        Map result = service.deleteGroups();
    +        assertNotNull(result.get(GROUP),
                 "Group was deleted successfully, but it shouldn't have been. Result was:(" + result + ")");
    -        assertTrue(result.size() == 1 && result.contains(GROUP) && result.get(GROUP).get().getCause() instanceof GroupNotEmptyException,
    +        assertTrue(result.size() == 1 && result.containsKey(GROUP) && result.get(GROUP).getCause() instanceof GroupNotEmptyException,
                 "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not detected while deleting consumer group. Result was:(" + result + ")");
         }
     
    @@ -139,21 +134,18 @@ public void testDeleteCmdEmptyGroup(String quorum) throws Exception {
             ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
     
             TestUtils.waitForCondition(
    -            () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state(), "Stable"),
    +            () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
                 "The group did not initialize as expected."
             );
     
             executor.shutdown();
     
             TestUtils.waitForCondition(
    -            () -> Objects.equals(service.collectGroupState(GROUP).state(), "Empty"),
    +            () -> Objects.equals(service.collectGroupState(GROUP).state, "Empty"),
                 "The group did not become empty as expected."
             );
     
    -        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
    -            service.deleteGroups();
    -            return null;
    -        });
    +        String output = ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
             assertTrue(output.contains("Deletion of requested consumer groups ('" + GROUP + "') was successful."),
                 "The consumer group could not be deleted as expected");
         }
    @@ -173,10 +165,10 @@ public void testDeleteCmdAllGroups(String quorum) throws Exception {
             ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
     
             TestUtils.waitForCondition(() ->
    -            Objects.equals(service.listConsumerGroups().toSet(), set(groups.keySet())) &&
    +            new HashSet<>(service.listConsumerGroups()).equals(groups.keySet()) &&
                     groups.keySet().stream().allMatch(groupId -> {
                         try {
    -                        return Objects.equals(service.collectGroupState(groupId).state(), "Stable");
    +                        return Objects.equals(service.collectGroupState(groupId).state, "Stable");
                         } catch (Exception e) {
                             throw new RuntimeException(e);
                         }
    @@ -189,17 +181,14 @@ public void testDeleteCmdAllGroups(String quorum) throws Exception {
             TestUtils.waitForCondition(() ->
                 groups.keySet().stream().allMatch(groupId -> {
                     try {
    -                    return Objects.equals(service.collectGroupState(groupId).state(), "Empty");
    +                    return Objects.equals(service.collectGroupState(groupId).state, "Empty");
                     } catch (Exception e) {
                         throw new RuntimeException(e);
                     }
                 }),
                 "The group did not become empty as expected.");
     
    -        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
    -            service.deleteGroups();
    -            return null;
    -        }).trim();
    +        String output = ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim();
             Set expectedGroupsForDeletion = groups.keySet();
             Set deletedGroupsGrepped = Arrays.stream(output.substring(output.indexOf('(') + 1, output.indexOf(')')).split(","))
                 .map(str -> str.replaceAll("'", "").trim()).collect(Collectors.toSet());
    @@ -220,17 +209,17 @@ public void testDeleteEmptyGroup(String quorum) throws Exception {
             ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
     
             TestUtils.waitForCondition(
    -            () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state(), "Stable"),
    +            () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
                 "The group did not initialize as expected.");
     
             executor.shutdown();
     
             TestUtils.waitForCondition(
    -            () -> Objects.equals(service.collectGroupState(GROUP).state(), "Empty"),
    +            () -> Objects.equals(service.collectGroupState(GROUP).state, "Empty"),
                 "The group did not become empty as expected.");
     
    -        scala.collection.Map result = service.deleteGroups();
    -        assertTrue(result.size() == 1 && result.contains(GROUP) && result.get(GROUP).get() == null,
    +        Map result = service.deleteGroups();
    +        assertTrue(result.size() == 1 && result.containsKey(GROUP) && result.get(GROUP) == null,
                 "The consumer group could not be deleted as expected");
         }
     
    @@ -246,23 +235,20 @@ public void testDeleteCmdWithMixOfSuccessAndError(String quorum) throws Exceptio
             ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
     
             TestUtils.waitForCondition(
    -            () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state(), "Stable"),
    +            () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
                 "The group did not initialize as expected.");
     
             executor.shutdown();
     
             TestUtils.waitForCondition(
    -            () -> Objects.equals(service.collectGroupState(GROUP).state(), "Empty"),
    +            () -> Objects.equals(service.collectGroupState(GROUP).state, "Empty"),
                 "The group did not become empty as expected.");
     
             cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--group", missingGroup};
     
             ConsumerGroupCommand.ConsumerGroupService service2 = getConsumerGroupService(cgcArgs);
     
    -        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
    -            service2.deleteGroups();
    -            return null;
    -        });
    +        String output = ToolsTestUtils.grabConsoleOutput(service2::deleteGroups);
             assertTrue(output.contains("Group '" + missingGroup + "' could not be deleted due to:")
                 && output.contains(Errors.GROUP_ID_NOT_FOUND.message())
                 && output.contains("These consumer groups were deleted successfully: '" + GROUP + "'"),
    @@ -281,23 +267,23 @@ public void testDeleteWithMixOfSuccessAndError(String quorum) throws Exception {
             ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
     
             TestUtils.waitForCondition(
    -            () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state(), "Stable"),
    +            () -> service.listConsumerGroups().contains(GROUP) && Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
                 "The group did not initialize as expected.");
     
             executor.shutdown();
     
             TestUtils.waitForCondition(
    -            () -> Objects.equals(service.collectGroupState(GROUP).state(), "Empty"),
    +            () -> Objects.equals(service.collectGroupState(GROUP).state, "Empty"),
                 "The group did not become empty as expected.");
     
             cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--group", missingGroup};
     
             ConsumerGroupCommand.ConsumerGroupService service2 = getConsumerGroupService(cgcArgs);
    -        scala.collection.Map result = service2.deleteGroups();
    +        Map result = service2.deleteGroups();
             assertTrue(result.size() == 2 &&
    -                result.contains(GROUP) && result.get(GROUP).get() == null &&
    -                result.contains(missingGroup) &&
    -                result.get(missingGroup).get().getMessage().contains(Errors.GROUP_ID_NOT_FOUND.message()),
    +                result.containsKey(GROUP) && result.get(GROUP) == null &&
    +                result.containsKey(missingGroup) &&
    +                result.get(missingGroup).getMessage().contains(Errors.GROUP_ID_NOT_FOUND.message()),
                 "The consumer group deletion did not work as expected");
         }
     
    diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java
    index 2ac093923c..292180b8af 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java
    @@ -16,7 +16,6 @@
      */
     package org.apache.kafka.tools.consumer.group;
     
    -import kafka.admin.ConsumerGroupCommand;
     import kafka.utils.TestUtils;
     import org.apache.kafka.clients.consumer.Consumer;
     import org.apache.kafka.clients.consumer.ConsumerConfig;
    @@ -34,6 +33,8 @@
     import org.junit.jupiter.params.provider.ValueSource;
     
     import java.util.Collections;
    +import java.util.Map;
    +import java.util.Map.Entry;
     import java.util.Properties;
     import java.util.concurrent.ExecutionException;
     
    @@ -59,8 +60,8 @@ public void testDeleteOffsetsNonExistingGroup() {
             String topic = "foo:1";
             ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(getArgs(group, topic));
     
    -        scala.Tuple2> res = service.deleteOffsets(group, seq(Collections.singleton(topic)).toList());
    -        assertEquals(Errors.GROUP_ID_NOT_FOUND, res._1);
    +        Entry> res = service.deleteOffsets(group, Collections.singletonList(topic));
    +        assertEquals(Errors.GROUP_ID_NOT_FOUND, res.getKey());
         }
     
         @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
    @@ -144,18 +145,18 @@ private void testWithConsumerGroup(java.util.function.Consumer withCon
             withConsumerGroup.accept(() -> {
                 String topic = inputPartition >= 0 ? inputTopic + ":" + inputPartition : inputTopic;
                 ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(getArgs(GROUP, topic));
    -            scala.Tuple2> res = service.deleteOffsets(GROUP, seq(Collections.singletonList(topic)).toList());
    -            Errors topLevelError = res._1;
    -            scala.collection.Map partitions = res._2;
    +            Entry> res = service.deleteOffsets(GROUP, Collections.singletonList(topic));
    +            Errors topLevelError = res.getKey();
    +            Map partitions = res.getValue();
                 TopicPartition tp = new TopicPartition(inputTopic, expectedPartition);
                 // Partition level error should propagate to top level, unless this is due to a missed partition attempt.
                 if (inputPartition >= 0) {
                     assertEquals(expectedError, topLevelError);
                 }
                 if (expectedError == Errors.NONE)
    -                assertNull(partitions.get(tp).get());
    +                assertNull(partitions.get(tp));
                 else
    -                assertEquals(expectedError.exception(), partitions.get(tp).get().getCause());
    +                assertEquals(expectedError.exception(), partitions.get(tp).getCause());
             });
         }
     
    diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java
    new file mode 100644
    index 0000000000..4b5b0b93f5
    --- /dev/null
    +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java
    @@ -0,0 +1,848 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements. See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License. You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.kafka.tools.consumer.group;
    +
    +import org.apache.kafka.clients.consumer.ConsumerConfig;
    +import org.apache.kafka.clients.consumer.RangeAssignor;
    +import org.apache.kafka.clients.consumer.RoundRobinAssignor;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.errors.TimeoutException;
    +import org.apache.kafka.common.utils.AppInfoParser;
    +import org.apache.kafka.common.utils.Exit;
    +import org.apache.kafka.test.TestUtils;
    +import org.apache.kafka.tools.ToolsTestUtils;
    +import org.junit.jupiter.params.ParameterizedTest;
    +import org.junit.jupiter.params.provider.MethodSource;
    +import org.junit.jupiter.params.provider.ValueSource;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Map.Entry;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.concurrent.ExecutionException;
    +import java.util.concurrent.atomic.AtomicInteger;
    +import java.util.concurrent.atomic.AtomicReference;
    +import java.util.function.Predicate;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.kafka.test.TestUtils.RANDOM;
    +import static org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES;
    +import static org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME;
    +import static org.junit.jupiter.api.Assertions.assertEquals;
    +import static org.junit.jupiter.api.Assertions.assertInstanceOf;
    +import static org.junit.jupiter.api.Assertions.assertThrows;
    +import static org.junit.jupiter.api.Assertions.assertTrue;
    +import static org.junit.jupiter.api.Assertions.fail;
    +
    +public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest {
    +    private static final List> DESCRIBE_TYPE_OFFSETS = Arrays.asList(Collections.singletonList(""), Collections.singletonList("--offsets"));
    +    private static final List> DESCRIBE_TYPE_MEMBERS = Arrays.asList(Collections.singletonList("--members"), Arrays.asList("--members", "--verbose"));
    +    private static final List> DESCRIBE_TYPE_STATE = Collections.singletonList(Collections.singletonList("--state"));
    +    private static final List> DESCRIBE_TYPES;
    +
    +    static {
    +        List> describeTypes = new ArrayList<>();
    +
    +        describeTypes.addAll(DESCRIBE_TYPE_OFFSETS);
    +        describeTypes.addAll(DESCRIBE_TYPE_MEMBERS);
    +        describeTypes.addAll(DESCRIBE_TYPE_STATE);
    +
    +        DESCRIBE_TYPES = describeTypes;
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeNonExistingGroup(String quorum, String groupProtocol) {
    +        createOffsetsTopic(listenerName(), new Properties());
    +        String missingGroup = "missing.group";
    +
    +        for (List describeType : DESCRIBE_TYPES) {
    +            // note the group to be queried is a different (non-existing) group
    +            List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", missingGroup));
    +            cgcArgs.addAll(describeType);
    +            ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0]));
    +
    +            String output = ToolsTestUtils.grabConsoleOutput(describeGroups(service));
    +            assertTrue(output.contains("Consumer group '" + missingGroup + "' does not exist."),
    +                "Expected error was not detected for describe option '" + String.join(" ", describeType) + "'");
    +        }
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
    +    @ValueSource(strings = {"zk", "kraft"})
    +    public void testDescribeWithMultipleSubActions(String quorum) {
    +        AtomicInteger exitStatus = new AtomicInteger(0);
    +        AtomicReference exitMessage = new AtomicReference<>("");
    +        Exit.setExitProcedure((status, err) -> {
    +            exitStatus.set(status);
    +            exitMessage.set(err);
    +            throw new RuntimeException();
    +        });
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP, "--members", "--state"};
    +        try {
    +            assertThrows(RuntimeException.class, () -> ConsumerGroupCommand.main(cgcArgs));
    +        } finally {
    +            Exit.resetExitProcedure();
    +        }
    +        assertEquals(1, exitStatus.get());
    +        assertTrue(exitMessage.get().contains("Option [describe] takes at most one of these options"));
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
    +    @ValueSource(strings = {"zk", "kraft"})
    +    public void testDescribeWithStateValue(String quorum) {
    +        AtomicInteger exitStatus = new AtomicInteger(0);
    +        AtomicReference exitMessage = new AtomicReference<>("");
    +        Exit.setExitProcedure((status, err) -> {
    +            exitStatus.set(status);
    +            exitMessage.set(err);
    +            throw new RuntimeException();
    +        });
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--all-groups", "--state", "Stable"};
    +        try {
    +            assertThrows(RuntimeException.class, () -> ConsumerGroupCommand.main(cgcArgs));
    +        } finally {
    +            Exit.resetExitProcedure();
    +        }
    +        assertEquals(1, exitStatus.get());
    +        assertTrue(exitMessage.get().contains("Option [describe] does not take a value for [state]"));
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
    +    @ValueSource(strings = {"zk", "kraft"})
    +    public void testPrintVersion(String quorum) {
    +        ToolsTestUtils.MockExitProcedure exitProcedure = new ToolsTestUtils.MockExitProcedure();
    +        Exit.setExitProcedure(exitProcedure);
    +        try {
    +            String out = ToolsTestUtils.captureStandardOut(() -> ConsumerGroupCommandOptions.fromArgs(new String[]{"--version"}));
    +            assertEquals(0, exitProcedure.statusCode());
    +            assertEquals(AppInfoParser.getVersion(), out);
    +        } finally {
    +            Exit.resetExitProcedure();
    +        }
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeOffsetsOfNonExistingGroup(String quorum, String groupProtocol) throws Exception {
    +        String group = "missing.group";
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // run one consumer in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(1, groupProtocol);
    +        // note the group to be queried is a different (non-existing) group
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        Entry, Optional>> res = service.collectGroupOffsets(group);
    +        assertTrue(res.getKey().map(s -> s.contains("Dead")).orElse(false) && res.getValue().map(Collection::isEmpty).orElse(false),
    +            "Expected the state to be 'Dead', with no members in the group '" + group + "'.");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeMembersOfNonExistingGroup(String quorum, String groupProtocol) throws Exception {
    +        String group = "missing.group";
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // run one consumer in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(1, groupProtocol);
    +        // note the group to be queried is a different (non-existing) group
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        Entry, Optional>> res = service.collectGroupMembers(group, false);
    +        assertTrue(res.getKey().map(s -> s.contains("Dead")).orElse(false) && res.getValue().map(Collection::isEmpty).orElse(false),
    +            "Expected the state to be 'Dead', with no members in the group '" + group + "'.");
    +
    +        Entry, Optional>> res2 = service.collectGroupMembers(group, true);
    +        assertTrue(res2.getKey().map(s -> s.contains("Dead")).orElse(false) && res2.getValue().map(Collection::isEmpty).orElse(false),
    +            "Expected the state to be 'Dead', with no members in the group '" + group + "' (verbose option).");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeStateOfNonExistingGroup(String quorum, String groupProtocol) throws Exception {
    +        String group = "missing.group";
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // run one consumer in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(1, groupProtocol);
    +        // note the group to be queried is a different (non-existing) group
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        GroupState state = service.collectGroupState(group);
    +        assertTrue(Objects.equals(state.state, "Dead") && state.numMembers == 0 &&
    +                state.coordinator != null && !brokers().filter(s -> s.config().brokerId() == state.coordinator.id()).isEmpty(),
    +            "Expected the state to be 'Dead', with no members in the group '" + group + "'."
    +        );
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeExistingGroup(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        for (List describeType : DESCRIBE_TYPES) {
    +            String group = GROUP + String.join("", describeType);
    +            // run one consumer in the group consuming from a single-partition topic
    +            addConsumerGroupExecutor(1, TOPIC, group, groupProtocol);
    +            List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group));
    +            cgcArgs.addAll(describeType);
    +            ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0]));
    +
    +            TestUtils.waitForCondition(() -> {
    +                Entry res = ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
    +                return res.getKey().trim().split("\n").length == 2 && res.getValue().isEmpty();
    +            }, "Expected a data row and no error in describe results with describe type " + String.join(" ", describeType) + ".");
    +        }
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeExistingGroups(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // Create N single-threaded consumer groups from a single-partition topic
    +        List groups = new ArrayList<>();
    +
    +        for (List describeType : DESCRIBE_TYPES) {
    +            String group = GROUP + String.join("", describeType);
    +            addConsumerGroupExecutor(1, TOPIC, group, groupProtocol);
    +            groups.addAll(Arrays.asList("--group", group));
    +        }
    +
    +        int expectedNumLines = DESCRIBE_TYPES.size() * 2;
    +
    +        for (List describeType : DESCRIBE_TYPES) {
    +            List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe"));
    +            cgcArgs.addAll(groups);
    +            cgcArgs.addAll(describeType);
    +            ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0]));
    +
    +            TestUtils.waitForCondition(() -> {
    +                Entry res = ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
    +                long numLines = Arrays.stream(res.getKey().trim().split("\n")).filter(line -> !line.isEmpty()).count();
    +                return (numLines == expectedNumLines) && res.getValue().isEmpty();
    +            }, "Expected a data row and no error in describe results with describe type " + String.join(" ", describeType) + ".");
    +        }
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeAllExistingGroups(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // Create N single-threaded consumer groups from a single-partition topic
    +        for (List describeType : DESCRIBE_TYPES) {
    +            String group = GROUP + String.join("", describeType);
    +            addConsumerGroupExecutor(1, TOPIC, group, groupProtocol);
    +        }
    +
    +        int expectedNumLines = DESCRIBE_TYPES.size() * 2;
    +
    +        for (List describeType : DESCRIBE_TYPES) {
    +            List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--all-groups"));
    +            cgcArgs.addAll(describeType);
    +            ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0]));
    +
    +            TestUtils.waitForCondition(() -> {
    +                Entry res = ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
    +                long numLines = Arrays.stream(res.getKey().trim().split("\n")).filter(s -> !s.isEmpty()).count();
    +                return (numLines == expectedNumLines) && res.getValue().isEmpty();
    +            }, "Expected a data row and no error in describe results with describe type " + String.join(" ", describeType) + ".");
    +        }
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeOffsetsOfExistingGroup(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // run one consumer in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(1, groupProtocol);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            Entry, Optional>> groupOffsets = service.collectGroupOffsets(GROUP);
    +            Optional state = groupOffsets.getKey();
    +            Optional> assignments = groupOffsets.getValue();
    +
    +            Predicate isGrp = s -> Objects.equals(s.group, GROUP);
    +
    +            boolean res = state.map(s -> s.contains("Stable")).orElse(false) &&
    +                assignments.isPresent() &&
    +                assignments.get().stream().filter(isGrp).count() == 1;
    +
    +            if (!res)
    +                return false;
    +
    +            Optional maybePartitionState = assignments.get().stream().filter(isGrp).findFirst();
    +            if (!maybePartitionState.isPresent())
    +                return false;
    +
    +            PartitionAssignmentState partitionState = maybePartitionState.get();
    +
    +            return !partitionState.consumerId.map(s0 -> s0.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) &&
    +                !partitionState.clientId.map(s0 -> s0.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) &&
    +                !partitionState.host.map(h -> h.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false);
    +        }, "Expected a 'Stable' group status, rows and valid values for consumer id / client id / host columns in describe results for group " + GROUP + ".");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeMembersOfExistingGroup(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // run one consumer in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(1, groupProtocol);
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            Entry, Optional>> groupMembers = service.collectGroupMembers(GROUP, false);
    +            Optional state = groupMembers.getKey();
    +            Optional> assignments = groupMembers.getValue();
    +
    +            Predicate isGrp = s -> Objects.equals(s.group, GROUP);
    +
    +            boolean res = state.map(s -> s.contains("Stable")).orElse(false) &&
    +                assignments.isPresent() &&
    +                assignments.get().stream().filter(s -> Objects.equals(s.group, GROUP)).count() == 1;
    +
    +            if (!res)
    +                return false;
    +
    +            Optional maybeAssignmentState = assignments.get().stream().filter(isGrp).findFirst();
    +            if (!maybeAssignmentState.isPresent())
    +                return false;
    +
    +            MemberAssignmentState assignmentState = maybeAssignmentState.get();
    +
    +            return !Objects.equals(assignmentState.consumerId, ConsumerGroupCommand.MISSING_COLUMN_VALUE) &&
    +                !Objects.equals(assignmentState.clientId, ConsumerGroupCommand.MISSING_COLUMN_VALUE) &&
    +                !Objects.equals(assignmentState.host, ConsumerGroupCommand.MISSING_COLUMN_VALUE);
    +        }, "Expected a 'Stable' group status, rows and valid member information for group " + GROUP + ".");
    +
    +        Entry, Optional>> res = service.collectGroupMembers(GROUP, true);
    +
    +        if (res.getValue().isPresent()) {
    +            assertTrue(res.getValue().get().size() == 1 && res.getValue().get().iterator().next().assignment.size() == 1,
    +                "Expected a topic partition assigned to the single group member for group " + GROUP);
    +        } else {
    +            fail("Expected partition assignments for members of group " + GROUP);
    +        }
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeStateOfExistingGroup(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // run one consumer in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(
    +            1,
    +            groupProtocol,
    +            // This is only effective when new protocol is used.
    +            Optional.of("range")
    +        );
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            GroupState state = service.collectGroupState(GROUP);
    +            return Objects.equals(state.state, "Stable") &&
    +                state.numMembers == 1 &&
    +                Objects.equals(state.assignmentStrategy, "range") &&
    +                state.coordinator != null &&
    +                brokers().count(s -> s.config().brokerId() == state.coordinator.id()) > 0;
    +        }, "Expected a 'Stable' group status, with one member and round robin assignment strategy for group " + GROUP + ".");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeStateOfExistingGroupWithNonDefaultAssignor(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // run one consumer in the group consuming from a single-partition topic
    +        String expectedName;
    +        if (groupProtocol.equals("consumer")) {
    +            addConsumerGroupExecutor(1, groupProtocol, Optional.of("range"));
    +            expectedName = "range";
    +        } else {
    +            addConsumerGroupExecutor(1, TOPIC, GROUP, RoundRobinAssignor.class.getName(), Optional.empty(), Optional.empty(), false, groupProtocol);
    +            expectedName = "roundrobin";
    +        }
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            GroupState state = service.collectGroupState(GROUP);
    +            return Objects.equals(state.state, "Stable") &&
    +                state.numMembers == 1 &&
    +                Objects.equals(state.assignmentStrategy, expectedName) &&
    +                state.coordinator != null &&
    +                brokers().count(s -> s.config().brokerId() == state.coordinator.id()) > 0;
    +        }, "Expected a 'Stable' group status, with one member and " + expectedName + " assignment strategy for group " + GROUP + ".");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeExistingGroupWithNoMembers(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        for (List describeType : DESCRIBE_TYPES) {
    +            String group = GROUP + String.join("", describeType);
    +            // run one consumer in the group consuming from a single-partition topic
    +            ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, group, groupProtocol);
    +            List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group));
    +            cgcArgs.addAll(describeType);
    +            ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0]));
    +
    +            TestUtils.waitForCondition(() -> {
    +                Entry res = ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
    +                return res.getKey().trim().split("\n").length == 2 && res.getValue().isEmpty();
    +            }, "Expected describe group results with one data row for describe type '" + String.join(" ", describeType) + "'");
    +
    +            // stop the consumer so the group has no active member anymore
    +            executor.shutdown();
    +            TestUtils.waitForCondition(
    +                () -> ToolsTestUtils.grabConsoleError(describeGroups(service)).contains("Consumer group '" + group + "' has no active members."),
    +                "Expected no active member in describe group results with describe type " + String.join(" ", describeType));
    +        }
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeOffsetsOfExistingGroupWithNoMembers(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // run one consumer in the group consuming from a single-partition topic
    +        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, GROUP, RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), true, groupProtocol);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            Entry, Optional>> res = service.collectGroupOffsets(GROUP);
    +            return res.getKey().map(s -> s.contains("Stable")).orElse(false)
    +                && res.getValue().map(c -> c.stream().anyMatch(assignment -> Objects.equals(assignment.group, GROUP) && assignment.offset.isPresent())).orElse(false);
    +        }, "Expected the group to initially become stable, and to find group in assignments after initial offset commit.");
    +
    +        // stop the consumer so the group has no active member anymore
    +        executor.shutdown();
    +
    +        TestUtils.waitForCondition(() -> {
    +            Entry, Optional>> offsets = service.collectGroupOffsets(GROUP);
    +            Optional state = offsets.getKey();
    +            Optional> assignments = offsets.getValue();
    +            List testGroupAssignments = assignments.get().stream().filter(a -> Objects.equals(a.group, GROUP)).collect(Collectors.toList());
    +            PartitionAssignmentState assignment = testGroupAssignments.get(0);
    +            return state.map(s -> s.contains("Empty")).orElse(false) &&
    +                testGroupAssignments.size() == 1 &&
    +                assignment.consumerId.map(c -> c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) && // the member should be gone
    +                assignment.clientId.map(c -> c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) &&
    +                assignment.host.map(c -> c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false);
    +        }, "failed to collect group offsets");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeMembersOfExistingGroupWithNoMembers(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // run one consumer in the group consuming from a single-partition topic
    +        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, groupProtocol);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            Entry, Optional>> res = service.collectGroupMembers(GROUP, false);
    +            return res.getKey().map(s -> s.contains("Stable")).orElse(false)
    +                && res.getValue().map(c -> c.stream().anyMatch(m -> Objects.equals(m.group, GROUP))).orElse(false);
    +        }, "Expected the group to initially become stable, and to find group in assignments after initial offset commit.");
    +
    +        // stop the consumer so the group has no active member anymore
    +        executor.shutdown();
    +
    +        TestUtils.waitForCondition(() -> {
    +            Entry, Optional>> res = service.collectGroupMembers(GROUP, false);
    +            return res.getKey().map(s -> s.contains("Empty")).orElse(false) && res.getValue().isPresent() && res.getValue().get().isEmpty();
    +        }, "Expected no member in describe group members results for group '" + GROUP + "'");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeStateOfExistingGroupWithNoMembers(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // run one consumer in the group consuming from a single-partition topic
    +        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, groupProtocol);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            GroupState state = service.collectGroupState(GROUP);
    +            return Objects.equals(state.state, "Stable") &&
    +                state.numMembers == 1 &&
    +                state.coordinator != null &&
    +                brokers().count(s -> s.config().brokerId() == state.coordinator.id()) > 0;
    +        }, "Expected the group '" + GROUP + "' to initially become stable, and have a single member.");
    +
    +        // stop the consumer so the group has no active member anymore
    +        executor.shutdown();
    +
    +        TestUtils.waitForCondition(() -> {
    +            GroupState state = service.collectGroupState(GROUP);
    +            return Objects.equals(state.state, "Empty") && state.numMembers == 0;
    +        }, "Expected the group '" + GROUP + "' to become empty after the only member leaving.");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeWithConsumersWithoutAssignedPartitions(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        for (List describeType : DESCRIBE_TYPES) {
    +            String group = GROUP + String.join("", describeType);
    +            // run two consumers in the group consuming from a single-partition topic
    +            addConsumerGroupExecutor(2, TOPIC, group, groupProtocol);
    +            List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group));
    +            cgcArgs.addAll(describeType);
    +            ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0]));
    +
    +            TestUtils.waitForCondition(() -> {
    +                Entry res = ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
    +                int expectedNumRows = DESCRIBE_TYPE_MEMBERS.contains(describeType) ? 3 : 2;
    +                return res.getValue().isEmpty() && res.getKey().trim().split("\n").length == expectedNumRows;
    +            }, "Expected a single data row in describe group result with describe type '" + String.join(" ", describeType) + "'");
    +        }
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeOffsetsWithConsumersWithoutAssignedPartitions(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // run two consumers in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(2, groupProtocol);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            Entry, Optional>> res = service.collectGroupOffsets(GROUP);
    +            return res.getKey().map(s -> s.contains("Stable")).isPresent() &&
    +                res.getValue().isPresent() &&
    +                res.getValue().get().stream().filter(s -> Objects.equals(s.group, GROUP)).count() == 1 &&
    +                res.getValue().get().stream().filter(x -> Objects.equals(x.group, GROUP) && x.partition.isPresent()).count() == 1;
    +        }, "Expected rows for consumers with no assigned partitions in describe group results");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeMembersWithConsumersWithoutAssignedPartitions(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // run two consumers in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(2, groupProtocol);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            Entry, Optional>> res = service.collectGroupMembers(GROUP, false);
    +            return res.getKey().map(s -> s.contains("Stable")).orElse(false) &&
    +                res.getValue().isPresent() &&
    +                res.getValue().get().stream().filter(s -> Objects.equals(s.group, GROUP)).count() == 2 &&
    +                res.getValue().get().stream().filter(x -> Objects.equals(x.group, GROUP) && x.numPartitions == 1).count() == 1 &&
    +                res.getValue().get().stream().filter(x -> Objects.equals(x.group, GROUP) && x.numPartitions == 0).count() == 1 &&
    +                res.getValue().get().stream().allMatch(s -> s.assignment.isEmpty());
    +        }, "Expected rows for consumers with no assigned partitions in describe group results");
    +
    +        Entry, Optional>> res = service.collectGroupMembers(GROUP, true);
    +        assertTrue(res.getKey().map(s -> s.contains("Stable")).orElse(false)
    +                && res.getValue().map(c -> c.stream().anyMatch(s -> !s.assignment.isEmpty())).orElse(false),
    +            "Expected additional columns in verbose version of describe members");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeStateWithConsumersWithoutAssignedPartitions(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        // run two consumers in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(2, groupProtocol);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            GroupState state = service.collectGroupState(GROUP);
    +            return Objects.equals(state.state, "Stable") && state.numMembers == 2;
    +        }, "Expected two consumers in describe group results");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeWithMultiPartitionTopicAndMultipleConsumers(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +        String topic2 = "foo2";
    +        createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties());
    +
    +        for (List describeType : DESCRIBE_TYPES) {
    +            String group = GROUP + String.join("", describeType);
    +            // run two consumers in the group consuming from a two-partition topic
    +            addConsumerGroupExecutor(2, topic2, group, groupProtocol);
    +            List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group));
    +            cgcArgs.addAll(describeType);
    +            ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0]));
    +
    +            TestUtils.waitForCondition(() -> {
    +                Entry res = ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
    +                int expectedNumRows = DESCRIBE_TYPE_STATE.contains(describeType) ? 2 : 3;
    +                return res.getValue().isEmpty() && res.getKey().trim().split("\n").length == expectedNumRows;
    +            }, "Expected a single data row in describe group result with describe type '" + String.join(" ", describeType) + "'");
    +        }
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeOffsetsWithMultiPartitionTopicAndMultipleConsumers(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +        String topic2 = "foo2";
    +        createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties());
    +
    +        // run two consumers in the group consuming from a two-partition topic
    +        addConsumerGroupExecutor(2, topic2, GROUP, groupProtocol);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            Entry, Optional>> res = service.collectGroupOffsets(GROUP);
    +            return res.getKey().map(s -> s.contains("Stable")).orElse(false) &&
    +                res.getValue().isPresent() &&
    +                res.getValue().get().stream().filter(s -> Objects.equals(s.group, GROUP)).count() == 2 &&
    +                res.getValue().get().stream().filter(x -> Objects.equals(x.group, GROUP) && x.partition.isPresent()).count() == 2 &&
    +                res.getValue().get().stream().noneMatch(x -> Objects.equals(x.group, GROUP) && !x.partition.isPresent());
    +        }, "Expected two rows (one row per consumer) in describe group results.");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeMembersWithMultiPartitionTopicAndMultipleConsumers(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +        String topic2 = "foo2";
    +        createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties());
    +
    +        // run two consumers in the group consuming from a two-partition topic
    +        addConsumerGroupExecutor(2, topic2, GROUP, groupProtocol);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            Entry, Optional>> res = service.collectGroupMembers(GROUP, false);
    +            return res.getKey().map(s -> s.contains("Stable")).orElse(false) &&
    +                res.getValue().isPresent() &&
    +                res.getValue().get().stream().filter(s -> Objects.equals(s.group, GROUP)).count() == 2 &&
    +                res.getValue().get().stream().filter(x -> Objects.equals(x.group, GROUP) && x.numPartitions == 1).count() == 2 &&
    +                res.getValue().get().stream().noneMatch(x -> Objects.equals(x.group, GROUP) && x.numPartitions == 0);
    +        }, "Expected two rows (one row per consumer) in describe group members results.");
    +
    +        Entry, Optional>> res = service.collectGroupMembers(GROUP, true);
    +        assertTrue(res.getKey().map(s -> s.contains("Stable")).orElse(false) && res.getValue().map(s -> s.stream().filter(x -> x.assignment.isEmpty()).count()).orElse(0L) == 0,
    +            "Expected additional columns in verbose version of describe members");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeStateWithMultiPartitionTopicAndMultipleConsumers(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +        String topic2 = "foo2";
    +        createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties());
    +
    +        // run two consumers in the group consuming from a two-partition topic
    +        addConsumerGroupExecutor(2, topic2, GROUP, groupProtocol);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            GroupState state = service.collectGroupState(GROUP);
    +            return Objects.equals(state.state, "Stable") && Objects.equals(state.group, GROUP) && state.numMembers == 2;
    +        }, "Expected a stable group with two members in describe group state result.");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
    +    @ValueSource(strings = {"zk", "kraft", "kraft+kip848"})
    +    public void testDescribeSimpleConsumerGroup(String quorum) throws Exception {
    +        // Ensure that the offsets of consumers which don't use group management are still displayed
    +
    +        createOffsetsTopic(listenerName(), new Properties());
    +        String topic2 = "foo2";
    +        createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties());
    +        addSimpleGroupExecutor(Arrays.asList(new TopicPartition(topic2, 0), new TopicPartition(topic2, 1)), GROUP);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            Entry, Optional>> res = service.collectGroupOffsets(GROUP);
    +            return res.getKey().map(s -> s.contains("Empty")).orElse(false)
    +                && res.getValue().isPresent() && res.getValue().get().stream().filter(s -> Objects.equals(s.group, GROUP)).count() == 2;
    +        }, "Expected a stable group with two members in describe group state result.");
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeGroupWithShortInitializationTimeout(String quorum, String groupProtocol) {
    +        // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't
    +        // complete before the timeout expires
    +
    +        List describeType = DESCRIBE_TYPES.get(RANDOM.nextInt(DESCRIBE_TYPES.size()));
    +        String group = GROUP + String.join("", describeType);
    +        // run one consumer in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(1, groupProtocol);
    +        // set the group initialization timeout too low for the group to stabilize
    +        List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--timeout", "1", "--group", group));
    +        cgcArgs.addAll(describeType);
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs.toArray(new String[0]));
    +
    +        ExecutionException e = assertThrows(ExecutionException.class, service::describeGroups);
    +        assertInstanceOf(TimeoutException.class, e.getCause());
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeGroupOffsetsWithShortInitializationTimeout(String quorum, String groupProtocol) {
    +        // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't
    +        // complete before the timeout expires
    +
    +        // run one consumer in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(1, groupProtocol);
    +
    +        // set the group initialization timeout too low for the group to stabilize
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP, "--timeout", "1"};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        Throwable e = assertThrows(ExecutionException.class, () -> service.collectGroupOffsets(GROUP));
    +        assertEquals(TimeoutException.class, e.getCause().getClass());
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeGroupMembersWithShortInitializationTimeout(String quorum, String groupProtocol) {
    +        // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't
    +        // complete before the timeout expires
    +
    +        // run one consumer in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(1, groupProtocol);
    +
    +        // set the group initialization timeout too low for the group to stabilize
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP, "--timeout", "1"};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        Throwable e = assertThrows(ExecutionException.class, () -> service.collectGroupMembers(GROUP, false));
    +        assertEquals(TimeoutException.class, e.getCause().getClass());
    +        e = assertThrows(ExecutionException.class, () -> service.collectGroupMembers(GROUP, true));
    +        assertEquals(TimeoutException.class, e.getCause().getClass());
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeGroupStateWithShortInitializationTimeout(String quorum, String groupProtocol) {
    +        // Let creation of the offsets topic happen during group initialization to ensure that initialization doesn't
    +        // complete before the timeout expires
    +
    +        // run one consumer in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(1, groupProtocol);
    +
    +        // set the group initialization timeout too low for the group to stabilize
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP, "--timeout", "1"};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        Throwable e = assertThrows(ExecutionException.class, () -> service.collectGroupState(GROUP));
    +        assertEquals(TimeoutException.class, e.getCause().getClass());
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
    +    @ValueSource(strings = {"zk", "kraft"})
    +    public void testDescribeWithUnrecognizedNewConsumerOption(String quorum) {
    +        String[] cgcArgs = new String[]{"--new-consumer", "--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        assertThrows(joptsimple.OptionException.class, () -> getConsumerGroupService(cgcArgs));
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
    +    public void testDescribeNonOffsetCommitGroup(String quorum, String groupProtocol) throws Exception {
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        Properties customProps = new Properties();
    +        // create a consumer group that never commits offsets
    +        customProps.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
    +        // run one consumer in the group consuming from a single-partition topic
    +        addConsumerGroupExecutor(1, TOPIC, GROUP, RangeAssignor.class.getName(), Optional.empty(), Optional.of(customProps), false, groupProtocol);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", GROUP};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        TestUtils.waitForCondition(() -> {
    +            Entry, Optional>> groupOffsets = service.collectGroupOffsets(GROUP);
    +
    +            Predicate isGrp = s -> Objects.equals(s.group, GROUP);
    +
    +            boolean res = groupOffsets.getKey().map(s -> s.contains("Stable")).orElse(false) &&
    +                groupOffsets.getValue().isPresent() &&
    +                groupOffsets.getValue().get().stream().filter(isGrp).count() == 1;
    +
    +            if (!res)
    +                return false;
    +
    +            Optional maybeAssignmentState = groupOffsets.getValue().get().stream().filter(isGrp).findFirst();
    +            if (!maybeAssignmentState.isPresent())
    +                return false;
    +
    +            PartitionAssignmentState assignmentState = maybeAssignmentState.get();
    +
    +            return assignmentState.consumerId.map(c -> !c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) &&
    +                assignmentState.clientId.map(c -> !c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) &&
    +                assignmentState.host.map(h -> !h.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false);
    +        }, "Expected a 'Stable' group status, rows and valid values for consumer id / client id / host columns in describe results for non-offset-committing group " + GROUP + ".");
    +    }
    +
    +    private Runnable describeGroups(ConsumerGroupCommand.ConsumerGroupService service) {
    +        return () -> {
    +            try {
    +                service.describeGroups();
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        };
    +    }
    +}
    diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java
    index 894f00df5e..6fd2928da9 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java
    @@ -17,87 +17,263 @@
     package org.apache.kafka.tools.consumer.group;
     
     import joptsimple.OptionException;
    -import kafka.admin.ConsumerGroupCommand;
     import org.apache.kafka.clients.admin.ConsumerGroupListing;
     import org.apache.kafka.common.ConsumerGroupState;
    +import org.apache.kafka.common.GroupType;
     import org.apache.kafka.test.TestUtils;
    +import org.junit.jupiter.api.Test;
    +import org.apache.kafka.tools.ToolsTestUtils;
     import org.junit.jupiter.params.ParameterizedTest;
    -import org.junit.jupiter.params.provider.ValueSource;
    +import org.junit.jupiter.params.provider.MethodSource;
     
     import java.util.Arrays;
    +import java.util.Collection;
     import java.util.Collections;
    +import java.util.EnumSet;
     import java.util.HashSet;
    -import java.util.List;
     import java.util.Objects;
     import java.util.Optional;
    +import java.util.Properties;
    +import java.util.List;
     import java.util.Set;
     import java.util.concurrent.atomic.AtomicReference;
     import java.util.stream.Collectors;
     
    +import static org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES;
     import static org.apache.kafka.common.utils.Utils.mkSet;
     import static org.junit.jupiter.api.Assertions.assertEquals;
     import static org.junit.jupiter.api.Assertions.assertThrows;
     
     public class ListConsumerGroupTest extends ConsumerGroupCommandTest {
    -    @ParameterizedTest
    -    @ValueSource(strings = {"zk", "kraft"})
    -    public void testListConsumerGroups(String quorum) throws Exception {
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource("getTestQuorumAndGroupProtocolParametersAll")
    +    public void testListConsumerGroupsWithoutFilters(String quorum, String groupProtocol) throws Exception {
             String simpleGroup = "simple-group";
    +
    +        createOffsetsTopic(listenerName(), new Properties());
    +
             addSimpleGroupExecutor(simpleGroup);
             addConsumerGroupExecutor(1);
    +        addConsumerGroupExecutor(1, PROTOCOL_GROUP, groupProtocol);
     
             String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list"};
             ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    -        scala.collection.Set expectedGroups = set(Arrays.asList(GROUP, simpleGroup));
    -        final AtomicReference foundGroups = new AtomicReference<>();
    +
    +        Set expectedGroups = set(Arrays.asList(GROUP, simpleGroup, PROTOCOL_GROUP));
    +        final AtomicReference foundGroups = new AtomicReference<>();
    +
             TestUtils.waitForCondition(() -> {
    -            foundGroups.set(service.listConsumerGroups().toSet());
    +            foundGroups.set(set(service.listConsumerGroups()));
                 return Objects.equals(expectedGroups, foundGroups.get());
             }, "Expected --list to show groups " + expectedGroups + ", but found " + foundGroups.get() + ".");
         }
     
    -    @ParameterizedTest
    -    @ValueSource(strings = {"zk", "kraft"})
    +    @Test
         public void testListWithUnrecognizedNewConsumerOption() {
             String[] cgcArgs = new String[]{"--new-consumer", "--bootstrap-server", bootstrapServers(listenerName()), "--list"};
             assertThrows(OptionException.class, () -> getConsumerGroupService(cgcArgs));
         }
     
    -    @ParameterizedTest
    -    @ValueSource(strings = {"zk", "kraft"})
    -    public void testListConsumerGroupsWithStates() throws Exception {
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource("getTestQuorumAndGroupProtocolParametersAll")
    +    public void testListConsumerGroupsWithStates(String quorum, String groupProtocol) throws Exception {
             String simpleGroup = "simple-group";
    +
    +        createOffsetsTopic(listenerName(), new Properties());
    +
             addSimpleGroupExecutor(simpleGroup);
    -        addConsumerGroupExecutor(1);
    +        addConsumerGroupExecutor(1, groupProtocol);
     
             String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state"};
             ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
     
    -        scala.collection.Set expectedListing = set(Arrays.asList(
    -            new ConsumerGroupListing(simpleGroup, true, Optional.of(ConsumerGroupState.EMPTY)),
    -            new ConsumerGroupListing(GROUP, false, Optional.of(ConsumerGroupState.STABLE))));
    +        Set expectedListing = mkSet(
    +            new ConsumerGroupListing(
    +                simpleGroup,
    +                true,
    +                Optional.of(ConsumerGroupState.EMPTY),
    +                Optional.of(GroupType.CLASSIC)
    +            ),
    +            new ConsumerGroupListing(
    +                GROUP,
    +                false,
    +                Optional.of(ConsumerGroupState.STABLE),
    +                Optional.of(GroupType.parse(groupProtocol))
    +            )
    +        );
     
    -        final AtomicReference foundListing = new AtomicReference<>();
    -        TestUtils.waitForCondition(() -> {
    -            foundListing.set(service.listConsumerGroupsWithState(set(Arrays.asList(ConsumerGroupState.values()))).toSet());
    -            return Objects.equals(expectedListing, foundListing.get());
    -        }, "Expected to show groups " + expectedListing + ", but found " + foundListing.get());
    +        assertGroupListing(
    +            service,
    +            Collections.emptySet(),
    +            EnumSet.allOf(ConsumerGroupState.class),
    +            expectedListing
    +        );
     
    -        scala.collection.Set expectedListingStable = set(Collections.singleton(
    -            new ConsumerGroupListing(GROUP, false, Optional.of(ConsumerGroupState.STABLE))));
    +        expectedListing = mkSet(
    +            new ConsumerGroupListing(
    +                GROUP,
    +                false,
    +                Optional.of(ConsumerGroupState.STABLE),
    +                Optional.of(GroupType.parse(groupProtocol))
    +            )
    +        );
     
    -        foundListing.set(null);
    +        assertGroupListing(
    +            service,
    +            Collections.emptySet(),
    +            mkSet(ConsumerGroupState.STABLE),
    +            expectedListing
    +        );
     
    -        TestUtils.waitForCondition(() -> {
    -            foundListing.set(service.listConsumerGroupsWithState(set(Collections.singleton(ConsumerGroupState.STABLE))).toSet());
    -            return Objects.equals(expectedListingStable, foundListing.get());
    -        }, "Expected to show groups " + expectedListingStable + ", but found " + foundListing.get());
    +        assertGroupListing(
    +            service,
    +            Collections.emptySet(),
    +            mkSet(ConsumerGroupState.PREPARING_REBALANCE),
    +            Collections.emptySet()
    +        );
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")
    +    public void testListConsumerGroupsWithTypesClassicProtocol(String quorum, String groupProtocol) throws Exception {
    +        String simpleGroup = "simple-group";
    +
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        addSimpleGroupExecutor(simpleGroup);
    +        addConsumerGroupExecutor(1);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list"};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        Set expectedListing = mkSet(
    +            new ConsumerGroupListing(
    +                simpleGroup,
    +                true,
    +                Optional.of(ConsumerGroupState.EMPTY),
    +                Optional.of(GroupType.CLASSIC)
    +            ),
    +            new ConsumerGroupListing(
    +                GROUP,
    +                false,
    +                Optional.of(ConsumerGroupState.STABLE),
    +                Optional.of(GroupType.CLASSIC)
    +            )
    +        );
    +
    +        // No filters explicitly mentioned. Expectation is that all groups are returned.
    +        assertGroupListing(
    +            service,
    +            Collections.emptySet(),
    +            Collections.emptySet(),
    +            expectedListing
    +        );
    +
    +        // When group type is mentioned:
    +        // Old Group Coordinator returns empty listings if the type is not Classic.
    +        // New Group Coordinator returns groups according to the filter.
    +        assertGroupListing(
    +            service,
    +            mkSet(GroupType.CONSUMER),
    +            Collections.emptySet(),
    +            Collections.emptySet()
    +        );
    +
    +        assertGroupListing(
    +            service,
    +            mkSet(GroupType.CLASSIC),
    +            Collections.emptySet(),
    +            expectedListing
    +        );
         }
     
    -    @ParameterizedTest
    -    @ValueSource(strings = {"zk", "kraft"})
    -    public void testConsumerGroupStatesFromString(String quorum) {
    -        scala.collection.Set result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable");
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource("getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly")
    +    public void testListConsumerGroupsWithTypesConsumerProtocol(String quorum, String groupProtocol) throws Exception {
    +        String simpleGroup = "simple-group";
    +
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        addSimpleGroupExecutor(simpleGroup);
    +        addConsumerGroupExecutor(1);
    +        addConsumerGroupExecutor(1, PROTOCOL_GROUP, groupProtocol);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list"};
    +        ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs);
    +
    +        // No filters explicitly mentioned. Expectation is that all groups are returned.
    +        Set expectedListing = mkSet(
    +            new ConsumerGroupListing(
    +                simpleGroup,
    +                true,
    +                Optional.of(ConsumerGroupState.EMPTY),
    +                Optional.of(GroupType.CLASSIC)
    +            ),
    +            new ConsumerGroupListing(
    +                GROUP,
    +                false,
    +                Optional.of(ConsumerGroupState.STABLE),
    +                Optional.of(GroupType.CLASSIC)
    +            ),
    +            new ConsumerGroupListing(
    +                PROTOCOL_GROUP,
    +                false,
    +                Optional.of(ConsumerGroupState.STABLE),
    +                Optional.of(GroupType.CONSUMER)
    +            )
    +        );
    +
    +        assertGroupListing(
    +            service,
    +            Collections.emptySet(),
    +            Collections.emptySet(),
    +            expectedListing
    +        );
    +
    +        // When group type is mentioned:
    +        // New Group Coordinator returns groups according to the filter.
    +        expectedListing = mkSet(
    +            new ConsumerGroupListing(
    +                PROTOCOL_GROUP,
    +                false,
    +                Optional.of(ConsumerGroupState.STABLE),
    +                Optional.of(GroupType.CONSUMER)
    +            )
    +        );
    +
    +        assertGroupListing(
    +            service,
    +            mkSet(GroupType.CONSUMER),
    +            Collections.emptySet(),
    +            expectedListing
    +        );
    +
    +        expectedListing = mkSet(
    +            new ConsumerGroupListing(
    +                simpleGroup,
    +                true,
    +                Optional.of(ConsumerGroupState.EMPTY),
    +                Optional.of(GroupType.CLASSIC)
    +            ),
    +            new ConsumerGroupListing(
    +                GROUP,
    +                false,
    +                Optional.of(ConsumerGroupState.STABLE),
    +                Optional.of(GroupType.CLASSIC)
    +            )
    +        );
    +
    +        assertGroupListing(
    +            service,
    +            mkSet(GroupType.CLASSIC),
    +            Collections.emptySet(),
    +            expectedListing
    +        );
    +    }
    +
    +    @Test
    +    public void testConsumerGroupStatesFromString() {
    +        Set result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable");
             assertEquals(set(Collections.singleton(ConsumerGroupState.STABLE)), result);
     
             result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable, PreparingRebalance");
    @@ -107,7 +283,7 @@ public void testConsumerGroupStatesFromString(String quorum) {
             assertEquals(set(Arrays.asList(ConsumerGroupState.DEAD, ConsumerGroupState.COMPLETING_REBALANCE)), result);
     
             result = ConsumerGroupCommand.consumerGroupStatesFromString("stable");
    -        assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE)), result);
    +        assertEquals(set(Collections.singletonList(ConsumerGroupState.STABLE)), result);
     
             result = ConsumerGroupCommand.consumerGroupStatesFromString("stable, assigning");
             assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE, ConsumerGroupState.ASSIGNING)), result);
    @@ -122,10 +298,31 @@ public void testConsumerGroupStatesFromString(String quorum) {
             assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupStatesFromString("   ,   ,"));
         }
     
    -    @ParameterizedTest
    -    @ValueSource(strings = {"zk", "kraft"})
    -    public void testListGroupCommand(String quorum) throws Exception {
    +    @Test
    +    public void testConsumerGroupTypesFromString() {
    +        Set result = ConsumerGroupCommand.consumerGroupTypesFromString("consumer");
    +        assertEquals(set(Collections.singleton(GroupType.CONSUMER)), result);
    +
    +        result = ConsumerGroupCommand.consumerGroupTypesFromString("consumer, classic");
    +        assertEquals(set(Arrays.asList(GroupType.CONSUMER, GroupType.CLASSIC)), result);
    +
    +        result = ConsumerGroupCommand.consumerGroupTypesFromString("Consumer, Classic");
    +        assertEquals(set(Arrays.asList(GroupType.CONSUMER, GroupType.CLASSIC)), result);
    +
    +        assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString("bad, wrong"));
    +
    +        assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString("  bad, generic"));
    +
    +        assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString("   ,   ,"));
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")
    +    public void testListGroupCommandClassicProtocol(String quorum, String groupProtocol) throws Exception {
             String simpleGroup = "simple-group";
    +
    +        createOffsetsTopic(listenerName(), new Properties());
    +
             addSimpleGroupExecutor(simpleGroup);
             addConsumerGroupExecutor(1);
     
    @@ -147,6 +344,24 @@ public void testListGroupCommand(String quorum) throws Exception {
                 )
             );
     
    +        validateListOutput(
    +            Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type"),
    +            Arrays.asList("GROUP", "TYPE"),
    +            mkSet(
    +                Arrays.asList(GROUP, "Classic"),
    +                Arrays.asList(simpleGroup, "Classic")
    +            )
    +        );
    +
    +        validateListOutput(
    +            Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "--state"),
    +            Arrays.asList("GROUP", "TYPE", "STATE"),
    +            mkSet(
    +                Arrays.asList(GROUP, "Classic", "Stable"),
    +                Arrays.asList(simpleGroup, "Classic", "Empty")
    +            )
    +        );
    +
             validateListOutput(
                 Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state", "Stable"),
                 Arrays.asList("GROUP", "STATE"),
    @@ -155,6 +370,7 @@ public void testListGroupCommand(String quorum) throws Exception {
                 )
             );
     
    +        // Check case-insensitivity in state filter.
             validateListOutput(
                 Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state", "stable"),
                 Arrays.asList("GROUP", "STATE"),
    @@ -162,6 +378,109 @@ public void testListGroupCommand(String quorum) throws Exception {
                     Arrays.asList(GROUP, "Stable")
                 )
             );
    +
    +        validateListOutput(
    +            Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "Classic"),
    +            Arrays.asList("GROUP", "TYPE"),
    +            mkSet(
    +                Arrays.asList(GROUP, "Classic"),
    +                Arrays.asList(simpleGroup, "Classic")
    +            )
    +        );
    +
    +        // Check case-insensitivity in type filter.
    +        validateListOutput(
    +            Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "classic"),
    +            Arrays.asList("GROUP", "TYPE"),
    +            mkSet(
    +                Arrays.asList(GROUP, "Classic"),
    +                Arrays.asList(simpleGroup, "Classic")
    +            )
    +        );
    +    }
    +
    +    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
    +    @MethodSource("getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly")
    +    public void testListGroupCommandConsumerProtocol(String quorum, String groupProtocol) throws Exception {
    +        String simpleGroup = "simple-group";
    +
    +        createOffsetsTopic(listenerName(), new Properties());
    +
    +        addSimpleGroupExecutor(simpleGroup);
    +        addConsumerGroupExecutor(1, PROTOCOL_GROUP, groupProtocol);
    +
    +        validateListOutput(
    +            Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list"),
    +            Collections.emptyList(),
    +            mkSet(
    +                Collections.singletonList(PROTOCOL_GROUP),
    +                Collections.singletonList(simpleGroup)
    +            )
    +        );
    +
    +        validateListOutput(
    +            Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state"),
    +            Arrays.asList("GROUP", "STATE"),
    +            mkSet(
    +                Arrays.asList(PROTOCOL_GROUP, "Stable"),
    +                Arrays.asList(simpleGroup, "Empty")
    +            )
    +        );
    +
    +        validateListOutput(
    +            Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type"),
    +            Arrays.asList("GROUP", "TYPE"),
    +            mkSet(
    +                Arrays.asList(PROTOCOL_GROUP, "Consumer"),
    +                Arrays.asList(simpleGroup, "Classic")
    +            )
    +        );
    +
    +        validateListOutput(
    +            Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "--state"),
    +            Arrays.asList("GROUP", "TYPE", "STATE"),
    +            mkSet(
    +                Arrays.asList(PROTOCOL_GROUP, "Consumer", "Stable"),
    +                Arrays.asList(simpleGroup, "Classic", "Empty")
    +            )
    +        );
    +
    +        validateListOutput(
    +            Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "consumer"),
    +            Arrays.asList("GROUP", "TYPE"),
    +            mkSet(
    +                Arrays.asList(PROTOCOL_GROUP, "Consumer")
    +            )
    +        );
    +
    +        validateListOutput(
    +            Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "consumer", "--state", "Stable"),
    +            Arrays.asList("GROUP", "TYPE", "STATE"),
    +            mkSet(
    +                Arrays.asList(PROTOCOL_GROUP, "Consumer", "Stable")
    +            )
    +        );
    +    }
    +
    +    /**
    +     * Validates the consumer group listings returned against expected values using specified filters.
    +     *
    +     * @param service           The service to list consumer groups.
    +     * @param typeFilterSet     Filters for group types, empty for no filter.
    +     * @param stateFilterSet    Filters for group states, empty for no filter.
    +     * @param expectedListing   Expected consumer group listings.
    +     */
    +    private static void assertGroupListing(
    +        ConsumerGroupCommand.ConsumerGroupService service,
    +        Set typeFilterSet,
    +        Set stateFilterSet,
    +        Set expectedListing
    +    ) throws Exception {
    +        final AtomicReference> foundListing = new AtomicReference<>();
    +        TestUtils.waitForCondition(() -> {
    +            foundListing.set(set(service.listConsumerGroupsWithFilters(set(typeFilterSet), set(stateFilterSet))));
    +            return Objects.equals(set(expectedListing), foundListing.get());
    +        }, () -> "Expected to show groups " + expectedListing + ", but found " + foundListing.get() + ".");
         }
     
         /**
    @@ -180,7 +499,7 @@ private static void validateListOutput(
         ) throws InterruptedException {
             final AtomicReference out = new AtomicReference<>("");
             TestUtils.waitForCondition(() -> {
    -            String output = runAndGrabConsoleOutput(args);
    +            String output = ToolsTestUtils.grabConsoleOutput(() -> ConsumerGroupCommand.main(args.toArray(new String[0])));
                 out.set(output);
     
                 int index = 0;
    @@ -204,12 +523,7 @@ private static void validateListOutput(
             }, () -> String.format("Expected header=%s and groups=%s, but found:%n%s", expectedHeader, expectedRows, out.get()));
         }
     
    -    private static String runAndGrabConsoleOutput(
    -        List args
    -    ) {
    -        return kafka.utils.TestUtils.grabConsoleOutput(() -> {
    -            ConsumerGroupCommand.main(args.toArray(new String[0]));
    -            return null;
    -        });
    +    public static  Set set(Collection set) {
    +        return new HashSet<>(set);
         }
     }
    diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java
    new file mode 100644
    index 0000000000..4d805ce68f
    --- /dev/null
    +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java
    @@ -0,0 +1,566 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements. See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License. You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.kafka.tools.consumer.group;
    +
    +import joptsimple.OptionException;
    +import org.apache.kafka.clients.consumer.Consumer;
    +import org.apache.kafka.clients.consumer.GroupProtocol;
    +import org.apache.kafka.clients.consumer.OffsetAndMetadata;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.test.TestUtils;
    +import org.junit.jupiter.api.Test;
    +
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.text.SimpleDateFormat;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Calendar;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Date;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Optional;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.function.Function;
    +import java.util.stream.Collectors;
    +import java.util.stream.IntStream;
    +
    +import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS;
    +import static org.junit.jupiter.api.Assertions.assertEquals;
    +import static org.junit.jupiter.api.Assertions.assertThrows;
    +import static org.junit.jupiter.api.Assertions.assertTrue;
    +
    +/**
    + * Test cases by:
    + * - Non-existing consumer group
    + * - One for each scenario, with scope=all-topics
    + * - scope=one topic, scenario=to-earliest
    + * - scope=one topic+partitions, scenario=to-earliest
    + * - scope=topics, scenario=to-earliest
    + * - scope=topics+partitions, scenario=to-earliest
    + * - export/import
    + */
    +public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest {
    +    private String[] basicArgs() {
    +        return new String[]{"--reset-offsets",
    +            "--bootstrap-server", bootstrapServers(listenerName()),
    +            "--timeout", Long.toString(DEFAULT_MAX_WAIT_MS)};
    +    }
    +
    +    private String[] buildArgsForGroups(List groups, String...args) {
    +        List res = new ArrayList<>(Arrays.asList(basicArgs()));
    +        for (String group : groups) {
    +            res.add("--group");
    +            res.add(group);
    +        }
    +        res.addAll(Arrays.asList(args));
    +        return res.toArray(new String[0]);
    +    }
    +
    +    private String[] buildArgsForGroup(String group, String...args) {
    +        return buildArgsForGroups(Collections.singletonList(group), args);
    +    }
    +
    +    private String[] buildArgsForAllGroups(String...args) {
    +        List res = new ArrayList<>(Arrays.asList(basicArgs()));
    +        res.add("--all-groups");
    +        res.addAll(Arrays.asList(args));
    +        return res.toArray(new String[0]);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsNotExistingGroup() throws Exception {
    +        String group = "missing.group";
    +        String[] args = buildArgsForGroup(group, "--all-topics", "--to-current", "--execute");
    +        ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args);
    +        // Make sure we got a coordinator
    +        TestUtils.waitForCondition(
    +            () -> Objects.equals(consumerGroupCommand.collectGroupState(group).coordinator.host(), "localhost"),
    +            "Can't find a coordinator");
    +        Map resetOffsets = consumerGroupCommand.resetOffsets().get(group);
    +        assertTrue(resetOffsets.isEmpty());
    +        assertTrue(committedOffsets(TOPIC, group).isEmpty());
    +    }
    +
    +    @Test
    +    public void testResetOffsetsExistingTopic() {
    +        String group = "new.group";
    +        String[] args = buildArgsForGroup(group, "--topic", TOPIC, "--to-offset", "50");
    +        produceMessages(TOPIC, 100);
    +        resetAndAssertOffsets(args, 50, true, Collections.singletonList(TOPIC));
    +        resetAndAssertOffsets(addTo(args, "--dry-run"), 50, true, Collections.singletonList(TOPIC));
    +        resetAndAssertOffsets(addTo(args, "--execute"), 50, false, Collections.singletonList(TOPIC));
    +    }
    +
    +    @Test
    +    public void testResetOffsetsExistingTopicSelectedGroups() throws Exception {
    +        produceMessages(TOPIC, 100);
    +        List groups = IntStream.rangeClosed(1, 3).mapToObj(id -> GROUP + id).collect(Collectors.toList());
    +        for (String group : groups) {
    +            ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, group, GroupProtocol.CLASSIC.name);
    +            awaitConsumerProgress(TOPIC, group, 100L);
    +            executor.shutdown();
    +        }
    +        String[] args = buildArgsForGroups(groups, "--topic", TOPIC, "--to-offset", "50");
    +        resetAndAssertOffsets(args, 50, true, Collections.singletonList(TOPIC));
    +        resetAndAssertOffsets(addTo(args, "--dry-run"), 50, true, Collections.singletonList(TOPIC));
    +        resetAndAssertOffsets(addTo(args, "--execute"), 50, false, Collections.singletonList(TOPIC));
    +    }
    +
    +    @Test
    +    public void testResetOffsetsExistingTopicAllGroups() throws Exception {
    +        String[] args = buildArgsForAllGroups("--topic", TOPIC, "--to-offset", "50");
    +        produceMessages(TOPIC, 100);
    +        for (int i = 1; i <= 3; i++) {
    +            String group = GROUP + i;
    +            ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, group, GroupProtocol.CLASSIC.name);
    +            awaitConsumerProgress(TOPIC, group, 100L);
    +            executor.shutdown();
    +        }
    +        resetAndAssertOffsets(args, 50, true, Collections.singletonList(TOPIC));
    +        resetAndAssertOffsets(addTo(args, "--dry-run"), 50, true, Collections.singletonList(TOPIC));
    +        resetAndAssertOffsets(addTo(args, "--execute"), 50, false, Collections.singletonList(TOPIC));
    +    }
    +
    +    @Test
    +    public void testResetOffsetsAllTopicsAllGroups() throws Exception {
    +        String[] args = buildArgsForAllGroups("--all-topics", "--to-offset", "50");
    +        List topics = IntStream.rangeClosed(1, 3).mapToObj(i -> TOPIC + i).collect(Collectors.toList());
    +        List groups = IntStream.rangeClosed(1, 3).mapToObj(i -> GROUP + i).collect(Collectors.toList());
    +        topics.forEach(topic -> produceMessages(topic, 100));
    +
    +        for (String topic : topics) {
    +            for (String group : groups) {
    +                ConsumerGroupExecutor executor = addConsumerGroupExecutor(3, topic, group, GroupProtocol.CLASSIC.name);
    +                awaitConsumerProgress(topic, group, 100);
    +                executor.shutdown();
    +            }
    +        }
    +        resetAndAssertOffsets(args, 50, true, topics);
    +        resetAndAssertOffsets(addTo(args, "--dry-run"), 50, true, topics);
    +        resetAndAssertOffsets(addTo(args, "--execute"), 50, false, topics);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsToLocalDateTime() throws Exception {
    +        SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS");
    +        Calendar calendar = Calendar.getInstance();
    +        calendar.add(Calendar.DATE, -1);
    +
    +        produceMessages(TOPIC, 100);
    +
    +        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, GROUP, GroupProtocol.CLASSIC.name);
    +        awaitConsumerProgress(TOPIC, GROUP, 100L);
    +        executor.shutdown();
    +
    +        String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-datetime", format.format(calendar.getTime()), "--execute");
    +        resetAndAssertOffsets(args, 0);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsToZonedDateTime() throws Exception {
    +        SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX");
    +
    +        produceMessages(TOPIC, 50);
    +        Date checkpoint = new Date();
    +        produceMessages(TOPIC, 50);
    +
    +        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, GROUP, GroupProtocol.CLASSIC.name);
    +        awaitConsumerProgress(TOPIC, GROUP, 100L);
    +        executor.shutdown();
    +
    +        String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-datetime", format.format(checkpoint), "--execute");
    +        resetAndAssertOffsets(args, 50);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsByDuration() throws Exception {
    +        String[] args = buildArgsForGroup(GROUP, "--all-topics", "--by-duration", "PT1M", "--execute");
    +        produceConsumeAndShutdown(TOPIC, GROUP, 100, 1);
    +        resetAndAssertOffsets(args, 0);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsByDurationToEarliest() throws Exception {
    +        String[] args = buildArgsForGroup(GROUP, "--all-topics", "--by-duration", "PT0.1S", "--execute");
    +        produceConsumeAndShutdown(TOPIC, GROUP, 100, 1);
    +        resetAndAssertOffsets(args, 100);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsByDurationFallbackToLatestWhenNoRecords() {
    +        String topic = "foo2";
    +        String[] args = buildArgsForGroup(GROUP, "--topic", topic, "--by-duration", "PT1M", "--execute");
    +        createTopic(topic, 1, 1, new Properties(), listenerName(), new Properties());
    +        resetAndAssertOffsets(args, 0, false, Collections.singletonList("foo2"));
    +
    +        adminZkClient().deleteTopic(topic);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsToEarliest() throws Exception {
    +        String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-earliest", "--execute");
    +        produceConsumeAndShutdown(TOPIC, GROUP, 100, 1);
    +        resetAndAssertOffsets(args, 0);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsToLatest() throws Exception {
    +        String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-latest", "--execute");
    +        produceConsumeAndShutdown(TOPIC, GROUP, 100, 1);
    +        produceMessages(TOPIC, 100);
    +        resetAndAssertOffsets(args, 200);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsToCurrentOffset() throws Exception {
    +        String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-current", "--execute");
    +        produceConsumeAndShutdown(TOPIC, GROUP, 100, 1);
    +        produceMessages(TOPIC, 100);
    +        resetAndAssertOffsets(args, 100);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsToSpecificOffset() throws Exception {
    +        String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-offset", "1", "--execute");
    +        produceConsumeAndShutdown(TOPIC, GROUP, 100, 1);
    +        resetAndAssertOffsets(args, 1);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsShiftPlus() throws Exception {
    +        String[] args = buildArgsForGroup(GROUP, "--all-topics", "--shift-by", "50", "--execute");
    +        produceConsumeAndShutdown(TOPIC, GROUP, 100, 1);
    +        produceMessages(TOPIC, 100);
    +        resetAndAssertOffsets(args, 150);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsShiftMinus() throws Exception {
    +        String[] args = buildArgsForGroup(GROUP, "--all-topics", "--shift-by", "-50", "--execute");
    +        produceConsumeAndShutdown(TOPIC, GROUP, 100, 1);
    +        produceMessages(TOPIC, 100);
    +        resetAndAssertOffsets(args, 50);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsShiftByLowerThanEarliest() throws Exception {
    +        String[] args = buildArgsForGroup(GROUP, "--all-topics", "--shift-by", "-150", "--execute");
    +        produceConsumeAndShutdown(TOPIC, GROUP, 100, 1);
    +        produceMessages(TOPIC, 100);
    +        resetAndAssertOffsets(args, 0);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsShiftByHigherThanLatest() throws Exception {
    +        String[] args = buildArgsForGroup(GROUP, "--all-topics", "--shift-by", "150", "--execute");
    +        produceConsumeAndShutdown(TOPIC, GROUP, 100, 1);
    +        produceMessages(TOPIC, 100);
    +        resetAndAssertOffsets(args, 200);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsToEarliestOnOneTopic() throws Exception {
    +        String[] args = buildArgsForGroup(GROUP, "--topic", TOPIC, "--to-earliest", "--execute");
    +        produceConsumeAndShutdown(TOPIC, GROUP, 100, 1);
    +        resetAndAssertOffsets(args, 0);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsToEarliestOnOneTopicAndPartition() throws Exception {
    +        String topic = "bar";
    +        createTopic(topic, 2, 1, new Properties(), listenerName(), new Properties());
    +
    +        String[] args = buildArgsForGroup(GROUP, "--topic", topic + ":1", "--to-earliest", "--execute");
    +        ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args);
    +
    +        produceConsumeAndShutdown(topic, GROUP, 100, 2);
    +        Map priorCommittedOffsets = committedOffsets(topic, GROUP);
    +
    +        TopicPartition tp0 = new TopicPartition(topic, 0);
    +        TopicPartition tp1 = new TopicPartition(topic, 1);
    +        Map expectedOffsets = new HashMap<>();
    +        expectedOffsets.put(tp0, priorCommittedOffsets.get(tp0));
    +        expectedOffsets.put(tp1, 0L);
    +        resetAndAssertOffsetsCommitted(consumerGroupCommand, expectedOffsets, topic);
    +
    +        adminZkClient().deleteTopic(topic);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsToEarliestOnTopics() throws Exception {
    +        String topic1 = "topic1";
    +        String topic2 = "topic2";
    +        createTopic(topic1, 1, 1, new Properties(), listenerName(), new Properties());
    +        createTopic(topic2, 1, 1, new Properties(), listenerName(), new Properties());
    +
    +        String[] args = buildArgsForGroup(GROUP, "--topic", topic1, "--topic", topic2, "--to-earliest", "--execute");
    +        ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args);
    +
    +        produceConsumeAndShutdown(topic1, GROUP, 100, 1);
    +        produceConsumeAndShutdown(topic2, GROUP, 100, 1);
    +
    +        TopicPartition tp1 = new TopicPartition(topic1, 0);
    +        TopicPartition tp2 = new TopicPartition(topic2, 0);
    +
    +        Map allResetOffsets = toOffsetMap(resetOffsets(consumerGroupCommand).get(GROUP));
    +        Map expMap = new HashMap<>();
    +        expMap.put(tp1, 0L);
    +        expMap.put(tp2, 0L);
    +        assertEquals(expMap, allResetOffsets);
    +        assertEquals(Collections.singletonMap(tp1, 0L), committedOffsets(topic1, GROUP));
    +        assertEquals(Collections.singletonMap(tp2, 0L), committedOffsets(topic2, GROUP));
    +
    +        adminZkClient().deleteTopic(topic1);
    +        adminZkClient().deleteTopic(topic2);
    +    }
    +
    +    @Test
    +    public void testResetOffsetsToEarliestOnTopicsAndPartitions() throws Exception {
    +        String topic1 = "topic1";
    +        String topic2 = "topic2";
    +
    +        createTopic(topic1, 2, 1, new Properties(), listenerName(), new Properties());
    +        createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties());
    +
    +        String[] args = buildArgsForGroup(GROUP, "--topic", topic1 + ":1", "--topic", topic2 + ":1", "--to-earliest", "--execute");
    +        ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args);
    +
    +        produceConsumeAndShutdown(topic1, GROUP, 100, 2);
    +        produceConsumeAndShutdown(topic2, GROUP, 100, 2);
    +
    +        Map priorCommittedOffsets1 = committedOffsets(topic1, GROUP);
    +        Map priorCommittedOffsets2 = committedOffsets(topic2, GROUP);
    +
    +        TopicPartition tp1 = new TopicPartition(topic1, 1);
    +        TopicPartition tp2 = new TopicPartition(topic2, 1);
    +        Map allResetOffsets = toOffsetMap(resetOffsets(consumerGroupCommand).get(GROUP));
    +        Map expMap = new HashMap<>();
    +        expMap.put(tp1, 0L);
    +        expMap.put(tp2, 0L);
    +        assertEquals(expMap, allResetOffsets);
    +        priorCommittedOffsets1.put(tp1, 0L);
    +        assertEquals(priorCommittedOffsets1, committedOffsets(topic1, GROUP));
    +        priorCommittedOffsets2.put(tp2, 0L);
    +        assertEquals(priorCommittedOffsets2, committedOffsets(topic2, GROUP));
    +
    +        adminZkClient().deleteTopic(topic1);
    +        adminZkClient().deleteTopic(topic2);
    +    }
    +
    +    @Test
    +    // This one deals with old CSV export/import format for a single --group arg: "topic,partition,offset" to support old behavior
    +    public void testResetOffsetsExportImportPlanSingleGroupArg() throws Exception {
    +        String topic = "bar";
    +        TopicPartition tp0 = new TopicPartition(topic, 0);
    +        TopicPartition tp1 = new TopicPartition(topic, 1);
    +        createTopic(topic, 2, 1, new Properties(), listenerName(), new Properties());
    +
    +        String[] cgcArgs = buildArgsForGroup(GROUP, "--all-topics", "--to-offset", "2", "--export");
    +        ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(cgcArgs);
    +
    +        produceConsumeAndShutdown(topic, GROUP, 100, 2);
    +
    +        File file = TestUtils.tempFile("reset", ".csv");
    +
    +        Map> exportedOffsets = consumerGroupCommand.resetOffsets();
    +        BufferedWriter bw = new BufferedWriter(new FileWriter(file));
    +        bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets));
    +        bw.close();
    +
    +        Map exp1 = new HashMap<>();
    +        exp1.put(tp0, 2L);
    +        exp1.put(tp1, 2L);
    +        assertEquals(exp1, toOffsetMap(exportedOffsets.get(GROUP)));
    +
    +        String[] cgcArgsExec = buildArgsForGroup(GROUP, "--all-topics", "--from-file", file.getCanonicalPath(), "--dry-run");
    +        ConsumerGroupCommand.ConsumerGroupService consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec);
    +        Map> importedOffsets = consumerGroupCommandExec.resetOffsets();
    +        assertEquals(exp1, toOffsetMap(importedOffsets.get(GROUP)));
    +
    +        adminZkClient().deleteTopic(topic);
    +    }
    +
    +    @Test
    +    // This one deals with universal CSV export/import file format "group,topic,partition,offset",
    +    // supporting multiple --group args or --all-groups arg
    +    public void testResetOffsetsExportImportPlan() throws Exception {
    +        String group1 = GROUP + "1";
    +        String group2 = GROUP + "2";
    +        String topic1 = "bar1";
    +        String topic2 = "bar2";
    +        TopicPartition t1p0 = new TopicPartition(topic1, 0);
    +        TopicPartition t1p1 = new TopicPartition(topic1, 1);
    +        TopicPartition t2p0 = new TopicPartition(topic2, 0);
    +        TopicPartition t2p1 = new TopicPartition(topic2, 1);
    +        createTopic(topic1, 2, 1, new Properties(), listenerName(), new Properties());
    +        createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties());
    +
    +        String[] cgcArgs = buildArgsForGroups(Arrays.asList(group1, group2), "--all-topics", "--to-offset", "2", "--export");
    +        ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(cgcArgs);
    +
    +        produceConsumeAndShutdown(topic1, group1, 100, 1);
    +        produceConsumeAndShutdown(topic2, group2, 100, 1);
    +
    +        awaitConsumerGroupInactive(consumerGroupCommand, group1);
    +        awaitConsumerGroupInactive(consumerGroupCommand, group2);
    +
    +        File file = TestUtils.tempFile("reset", ".csv");
    +
    +        Map> exportedOffsets = consumerGroupCommand.resetOffsets();
    +        BufferedWriter bw = new BufferedWriter(new FileWriter(file));
    +        bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets));
    +        bw.close();
    +        Map exp1 = new HashMap<>();
    +        exp1.put(t1p0, 2L);
    +        exp1.put(t1p1, 2L);
    +        Map exp2 = new HashMap<>();
    +        exp2.put(t2p0, 2L);
    +        exp2.put(t2p1, 2L);
    +
    +        assertEquals(exp1, toOffsetMap(exportedOffsets.get(group1)));
    +        assertEquals(exp2, toOffsetMap(exportedOffsets.get(group2)));
    +
    +        // Multiple --group's offset import
    +        String[] cgcArgsExec = buildArgsForGroups(Arrays.asList(group1, group2), "--all-topics", "--from-file", file.getCanonicalPath(), "--dry-run");
    +        ConsumerGroupCommand.ConsumerGroupService consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec);
    +        Map> importedOffsets = consumerGroupCommandExec.resetOffsets();
    +        assertEquals(exp1, toOffsetMap(importedOffsets.get(group1)));
    +        assertEquals(exp2, toOffsetMap(importedOffsets.get(group2)));
    +
    +        // Single --group offset import using "group,topic,partition,offset" csv format
    +        String[] cgcArgsExec2 = buildArgsForGroup(group1, "--all-topics", "--from-file", file.getCanonicalPath(), "--dry-run");
    +        ConsumerGroupCommand.ConsumerGroupService consumerGroupCommandExec2 = getConsumerGroupService(cgcArgsExec2);
    +        Map> importedOffsets2 = consumerGroupCommandExec2.resetOffsets();
    +        assertEquals(exp1, toOffsetMap(importedOffsets2.get(group1)));
    +
    +        adminZkClient().deleteTopic(TOPIC);
    +    }
    +
    +    @Test
    +    public void testResetWithUnrecognizedNewConsumerOption() {
    +        String[] cgcArgs = new String[]{"--new-consumer", "--bootstrap-server", bootstrapServers(listenerName()), "--reset-offsets",
    +            "--group", GROUP, "--all-topics", "--to-offset", "2", "--export"};
    +        assertThrows(OptionException.class, () -> getConsumerGroupService(cgcArgs));
    +    }
    +
    +    private void produceMessages(String topic, int numMessages) {
    +        List> records = IntStream.range(0, numMessages)
    +            .mapToObj(i -> new ProducerRecord(topic, new byte[100 * 1000]))
    +            .collect(Collectors.toList());
    +        kafka.utils.TestUtils.produceMessages(servers(), seq(records), 1);
    +    }
    +
    +    private void produceConsumeAndShutdown(String topic, String group, int totalMessages, int numConsumers) throws Exception {
    +        produceMessages(topic, totalMessages);
    +        ConsumerGroupExecutor executor = addConsumerGroupExecutor(numConsumers, topic, group, GroupProtocol.CLASSIC.name);
    +        awaitConsumerProgress(topic, group, totalMessages);
    +        executor.shutdown();
    +    }
    +
    +    private void awaitConsumerProgress(String topic,
    +                                       String group,
    +                                       long count) throws Exception {
    +        try (Consumer consumer = createNoAutoCommitConsumer(group)) {
    +            Set partitions = consumer.partitionsFor(topic).stream()
    +                .map(partitionInfo -> new TopicPartition(partitionInfo.topic(), partitionInfo.partition()))
    +                .collect(Collectors.toSet());
    +
    +            TestUtils.waitForCondition(() -> {
    +                Collection committed = consumer.committed(partitions).values();
    +                long total = committed.stream()
    +                    .mapToLong(offsetAndMetadata -> Optional.ofNullable(offsetAndMetadata).map(OffsetAndMetadata::offset).orElse(0L))
    +                    .sum();
    +
    +                return total == count;
    +            }, "Expected that consumer group has consumed all messages from topic/partition. " +
    +                "Expected offset: " + count + ". Actual offset: " + committedOffsets(topic, group).values().stream().mapToLong(Long::longValue).sum());
    +        }
    +    }
    +
    +    private void awaitConsumerGroupInactive(ConsumerGroupCommand.ConsumerGroupService consumerGroupService, String group) throws Exception {
    +        TestUtils.waitForCondition(() -> {
    +            String state = consumerGroupService.collectGroupState(group).state;
    +            return Objects.equals(state, "Empty") || Objects.equals(state, "Dead");
    +        }, "Expected that consumer group is inactive. Actual state: " + consumerGroupService.collectGroupState(group).state);
    +    }
    +
    +    private void resetAndAssertOffsets(String[] args,
    +                                       long expectedOffset) {
    +        resetAndAssertOffsets(args, expectedOffset, false, Collections.singletonList(TOPIC));
    +    }
    +
    +    private void resetAndAssertOffsets(String[] args,
    +                                      long expectedOffset,
    +                                      boolean dryRun,
    +                                      List topics) {
    +        ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args);
    +        Map> expectedOffsets = topics.stream().collect(Collectors.toMap(
    +            Function.identity(),
    +            topic -> Collections.singletonMap(new TopicPartition(topic, 0), expectedOffset)));
    +        Map> resetOffsetsResultByGroup = resetOffsets(consumerGroupCommand);
    +
    +        try {
    +            for (final String topic : topics) {
    +                resetOffsetsResultByGroup.forEach((group, partitionInfo) -> {
    +                    Map priorOffsets = committedOffsets(topic, group);
    +                    assertEquals(expectedOffsets.get(topic),
    +                        partitionInfo.entrySet().stream()
    +                            .filter(entry -> Objects.equals(entry.getKey().topic(), topic))
    +                            .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())));
    +                    assertEquals(dryRun ? priorOffsets : expectedOffsets.get(topic), committedOffsets(topic, group));
    +                });
    +            }
    +        } finally {
    +            consumerGroupCommand.close();
    +        }
    +    }
    +
    +    private void resetAndAssertOffsetsCommitted(ConsumerGroupCommand.ConsumerGroupService consumerGroupService,
    +                                                Map expectedOffsets,
    +                                                String topic) {
    +        Map> allResetOffsets = resetOffsets(consumerGroupService);
    +
    +        allResetOffsets.forEach((group, offsetsInfo) -> {
    +            offsetsInfo.forEach((tp, offsetMetadata) -> {
    +                assertEquals(offsetMetadata.offset(), expectedOffsets.get(tp));
    +                assertEquals(expectedOffsets, committedOffsets(topic, group));
    +            });
    +        });
    +    }
    +
    +    private Map> resetOffsets(ConsumerGroupCommand.ConsumerGroupService consumerGroupService) {
    +        return consumerGroupService.resetOffsets();
    +    }
    +
    +    Map toOffsetMap(Map map) {
    +        return map.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset()));
    +    }
    +
    +    private String[] addTo(String[] args, String...extra) {
    +        List res = new ArrayList<>(Arrays.asList(args));
    +        res.addAll(Arrays.asList(extra));
    +        return res.toArray(new String[0]);
    +    }
    +}
    diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java
    new file mode 100644
    index 0000000000..1ce1f909dd
    --- /dev/null
    +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements. See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License. You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.kafka.tools.consumer.group;
    +
    +import kafka.api.AbstractSaslTest;
    +import kafka.api.Both$;
    +import kafka.utils.JaasTestUtils;
    +import kafka.zk.ConfigEntityChangeNotificationZNode;
    +import org.apache.kafka.clients.admin.Admin;
    +import org.apache.kafka.clients.consumer.Consumer;
    +import org.apache.kafka.common.errors.SaslAuthenticationException;
    +import org.apache.kafka.common.security.auth.SecurityProtocol;
    +import org.apache.kafka.common.serialization.ByteArrayDeserializer;
    +import org.apache.kafka.test.TestUtils;
    +import org.junit.jupiter.api.AfterEach;
    +import org.junit.jupiter.api.BeforeEach;
    +import org.junit.jupiter.api.Test;
    +import org.junit.jupiter.api.TestInfo;
    +import org.junit.jupiter.api.function.Executable;
    +import scala.Option;
    +import scala.Some$;
    +import scala.collection.JavaConverters;
    +import scala.collection.Seq;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.time.Duration;
    +import java.util.Collections;
    +import java.util.Properties;
    +
    +import static org.apache.kafka.tools.consumer.group.ConsumerGroupCommandTest.seq;
    +import static org.junit.jupiter.api.Assertions.assertEquals;
    +import static org.junit.jupiter.api.Assertions.assertThrows;
    +import static org.junit.jupiter.api.Assertions.assertTrue;
    +
    +public class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest {
    +    private static final String TOPIC = "topic";
    +    public static final int NUM_PARTITIONS = 1;
    +    public static final int BROKER_COUNT = 1;
    +    public static final String KAFKA_CLIENT_SASL_MECHANISM = "SCRAM-SHA-256";
    +    private static final Seq KAFKA_SERVER_SASL_MECHANISMS = seq(Collections.singletonList(KAFKA_CLIENT_SASL_MECHANISM));
    +
    +    @SuppressWarnings({"deprecation"})
    +    private Consumer createConsumer() {
    +        return createConsumer(
    +            new ByteArrayDeserializer(),
    +            new ByteArrayDeserializer(),
    +            new Properties(),
    +            JavaConverters.asScalaSet(Collections.emptySet()).toList()
    +        );
    +    }
    +
    +    @Override
    +    public SecurityProtocol securityProtocol() {
    +        return SecurityProtocol.SASL_PLAINTEXT;
    +    }
    +
    +    @Override
    +    public Option serverSaslProperties() {
    +        return Some$.MODULE$.apply(kafkaServerSaslProperties(KAFKA_SERVER_SASL_MECHANISMS, KAFKA_CLIENT_SASL_MECHANISM));
    +    }
    +
    +    @Override
    +    public Option clientSaslProperties() {
    +        return Some$.MODULE$.apply(kafkaClientSaslProperties(KAFKA_CLIENT_SASL_MECHANISM, false));
    +    }
    +
    +    @Override
    +    public int brokerCount() {
    +        return 1;
    +    }
    +
    +    @Override
    +    public void configureSecurityBeforeServersStart(TestInfo testInfo) {
    +        super.configureSecurityBeforeServersStart(testInfo);
    +        zkClient().makeSurePersistentPathExists(ConfigEntityChangeNotificationZNode.path());
    +        // Create broker credentials before starting brokers
    +        createScramCredentials(zkConnect(), JaasTestUtils.KafkaScramAdmin(), JaasTestUtils.KafkaScramAdminPassword());
    +    }
    +
    +    @Override
    +    public Admin createPrivilegedAdminClient() {
    +        return createAdminClient(bootstrapServers(listenerName()), securityProtocol(), trustStoreFile(), clientSaslProperties(),
    +            KAFKA_CLIENT_SASL_MECHANISM, JaasTestUtils.KafkaScramAdmin(), JaasTestUtils.KafkaScramAdminPassword());
    +    }
    +
    +    @BeforeEach
    +    @Override
    +    public void setUp(TestInfo testInfo) {
    +        startSasl(jaasSections(KAFKA_SERVER_SASL_MECHANISMS, Some$.MODULE$.apply(KAFKA_CLIENT_SASL_MECHANISM), Both$.MODULE$,
    +            JaasTestUtils.KafkaServerContextName()));
    +        super.setUp(testInfo);
    +        createTopic(
    +            TOPIC,
    +            NUM_PARTITIONS,
    +            BROKER_COUNT,
    +            new Properties(),
    +            listenerName(),
    +            new Properties());
    +    }
    +
    +    @AfterEach
    +    @Override
    +    public void tearDown() {
    +        super.tearDown();
    +        closeSasl();
    +    }
    +
    +    @Test
    +    public void testConsumerGroupServiceWithAuthenticationFailure() throws Exception {
    +        ConsumerGroupCommand.ConsumerGroupService consumerGroupService = prepareConsumerGroupService();
    +        try (Consumer consumer = createConsumer()) {
    +            consumer.subscribe(Collections.singletonList(TOPIC));
    +
    +            verifyAuthenticationException(consumerGroupService::listGroups);
    +        } finally {
    +            consumerGroupService.close();
    +        }
    +    }
    +
    +    @Test
    +    public void testConsumerGroupServiceWithAuthenticationSuccess() throws Exception {
    +        createScramCredentialsViaPrivilegedAdminClient(JaasTestUtils.KafkaScramUser2(), JaasTestUtils.KafkaScramPassword2());
    +        ConsumerGroupCommand.ConsumerGroupService consumerGroupService = prepareConsumerGroupService();
    +        try (Consumer consumer = createConsumer()) {
    +            consumer.subscribe(Collections.singletonList(TOPIC));
    +
    +            TestUtils.waitForCondition(() -> {
    +                try {
    +                    consumer.poll(Duration.ofMillis(1000));
    +                    return true;
    +                } catch (SaslAuthenticationException ignored) {
    +                    return false;
    +                }
    +            }, "failed to poll data with authentication");
    +            assertEquals(1, consumerGroupService.listConsumerGroups().size());
    +        } finally {
    +            consumerGroupService.close();
    +        }
    +    }
    +
    +    private ConsumerGroupCommand.ConsumerGroupService prepareConsumerGroupService() throws IOException {
    +        File propsFile = TestUtils.tempFile(
    +            "security.protocol=SASL_PLAINTEXT\n" +
    +            "sasl.mechanism=" + KAFKA_CLIENT_SASL_MECHANISM);
    +
    +        String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()),
    +            "--describe",
    +            "--group", "test.group",
    +            "--command-config", propsFile.getAbsolutePath()};
    +        ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(cgcArgs);
    +        return new ConsumerGroupCommand.ConsumerGroupService(opts, Collections.emptyMap());
    +    }
    +
    +    private void verifyAuthenticationException(Executable action) {
    +        long startMs = System.currentTimeMillis();
    +        assertThrows(Exception.class, action);
    +        long elapsedMs = System.currentTimeMillis() - startMs;
    +        assertTrue(elapsedMs <= 5000, "Poll took too long, elapsed=" + elapsedMs);
    +    }
    +}
    diff --git a/tools/src/test/java/org/apache/kafka/tools/other/ReplicationQuotasTestRig.java b/tools/src/test/java/org/apache/kafka/tools/other/ReplicationQuotasTestRig.java
    index 618cf97299..c7c4a4f35e 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/other/ReplicationQuotasTestRig.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/other/ReplicationQuotasTestRig.java
    @@ -217,7 +217,7 @@ public void run(ExperimentDef config, Journal journal, boolean displayChartsOnSc
     
                 System.out.println("Generating Reassignment");
                 Map> newAssignment = ReassignPartitionsCommand.generateAssignment(adminClient,
    -                json(TOPIC_NAME), brokers.stream().map(Object::toString).collect(Collectors.joining(",")), true).v1;
    +                json(TOPIC_NAME), brokers.stream().map(Object::toString).collect(Collectors.joining(",")), true).getKey();
     
                 System.out.println("Starting Reassignment");
                 long start = System.currentTimeMillis();
    diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java
    index 83fc665e3e..0348aa24ff 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java
    @@ -43,7 +43,6 @@
     import org.apache.kafka.common.utils.Time;
     import org.apache.kafka.common.utils.Utils;
     import org.apache.kafka.tools.TerseException;
    -import org.apache.kafka.tools.Tuple2;
     import org.junit.jupiter.api.AfterEach;
     import org.junit.jupiter.api.Timeout;
     import org.junit.jupiter.params.ParameterizedTest;
    @@ -54,6 +53,7 @@
     import scala.collection.Seq;
     
     import java.io.Closeable;
    +import java.util.AbstractMap.SimpleImmutableEntry;
     import java.util.ArrayList;
     import java.util.Collection;
     import java.util.Collections;
    @@ -62,6 +62,7 @@
     import java.util.Iterator;
     import java.util.List;
     import java.util.Map;
    +import java.util.Map.Entry;
     import java.util.Objects;
     import java.util.Optional;
     import java.util.Properties;
    @@ -395,11 +396,11 @@ public void testCancellation(String quorum) throws Exception {
             waitForVerifyAssignment(cluster.adminClient, assignment, true,
                 new VerifyAssignmentResult(partStates, true, Collections.emptyMap(), false));
             // Cancel the reassignment.
    -        assertEquals(new Tuple2<>(new HashSet<>(asList(foo0, baz1)), Collections.emptySet()), runCancelAssignment(cluster.adminClient, assignment, true));
    +        assertEquals(new SimpleImmutableEntry<>(new HashSet<>(asList(foo0, baz1)), Collections.emptySet()), runCancelAssignment(cluster.adminClient, assignment, true));
             // Broker throttles are still active because we passed --preserve-throttles
             waitForInterBrokerThrottle(asList(0, 1, 2, 3), interBrokerThrottle);
             // Cancelling the reassignment again should reveal nothing to cancel.
    -        assertEquals(new Tuple2<>(Collections.emptySet(), Collections.emptySet()), runCancelAssignment(cluster.adminClient, assignment, false));
    +        assertEquals(new SimpleImmutableEntry<>(Collections.emptySet(), Collections.emptySet()), runCancelAssignment(cluster.adminClient, assignment, false));
             // This time, the broker throttles were removed.
             waitForBrokerLevelThrottles(unthrottledBrokerConfigs);
             // Verify that there are no ongoing reassignments.
    @@ -446,7 +447,7 @@ public void testCancellationWithAddingReplicaInIsr(String quorum) throws Excepti
             );
     
             // Now cancel the assignment and verify that the partition is removed from cancelled replicas
    -        assertEquals(new Tuple2<>(Collections.singleton(foo0), Collections.emptySet()), runCancelAssignment(cluster.adminClient, assignment, true));
    +        assertEquals(new SimpleImmutableEntry<>(Collections.singleton(foo0), Collections.emptySet()), runCancelAssignment(cluster.adminClient, assignment, true));
             verifyReplicaDeleted(foo0, 3);
             verifyReplicaDeleted(foo0, 4);
         }
    @@ -708,7 +709,7 @@ private void runExecuteAssignment(Admin adminClient,
             }
         }
     
    -    private Tuple2, Set> runCancelAssignment(
    +    private Entry, Set> runCancelAssignment(
             Admin adminClient,
             String jsonString,
             Boolean preserveThrottles
    diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java
    index 699e048fb2..c6f145d9a5 100644
    --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java
    +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java
    @@ -31,18 +31,19 @@
     import org.apache.kafka.common.utils.Time;
     import org.apache.kafka.server.common.AdminCommandFailedException;
     import org.apache.kafka.server.common.AdminOperationException;
    -import org.apache.kafka.tools.Tuple2;
     import org.junit.jupiter.api.AfterAll;
     import org.junit.jupiter.api.BeforeAll;
     import org.junit.jupiter.api.Test;
     import org.junit.jupiter.api.Timeout;
     
    +import java.util.AbstractMap.SimpleImmutableEntry;
     import java.util.ArrayList;
     import java.util.Collections;
     import java.util.HashMap;
     import java.util.HashSet;
     import java.util.List;
     import java.util.Map;
    +import java.util.Map.Entry;
     import java.util.Optional;
     import java.util.concurrent.ExecutionException;
     import java.util.stream.Collectors;
    @@ -174,14 +175,14 @@ public void testFindPartitionReassignmentStates() throws Exception {
                 expStates.put(new TopicPartition("foo", 1),
                     new PartitionReassignmentState(asList(1, 2, 3), asList(1, 2, 3), true));
     
    -            Tuple2, Boolean> actual =
    +            Entry, Boolean> actual =
                     findPartitionReassignmentStates(adminClient, asList(
    -                    new Tuple2<>(new TopicPartition("foo", 0), asList(0, 1, 3)),
    -                    new Tuple2<>(new TopicPartition("foo", 1), asList(1, 2, 3))
    +                    new SimpleImmutableEntry<>(new TopicPartition("foo", 0), asList(0, 1, 3)),
    +                    new SimpleImmutableEntry<>(new TopicPartition("foo", 1), asList(1, 2, 3))
                     ));
     
    -            assertEquals(expStates, actual.v1);
    -            assertTrue(actual.v2);
    +            assertEquals(expStates, actual.getKey());
    +            assertTrue(actual.getValue());
     
                 // Cancel the reassignment and test findPartitionReassignmentStates again.
                 Map cancelResult = cancelPartitionReassignments(adminClient,
    @@ -198,12 +199,12 @@ public void testFindPartitionReassignmentStates() throws Exception {
                     new PartitionReassignmentState(asList(1, 2, 3), asList(1, 2, 3), true));
     
                 actual = findPartitionReassignmentStates(adminClient, asList(
    -                new Tuple2<>(new TopicPartition("foo", 0), asList(0, 1, 3)),
    -                new Tuple2<>(new TopicPartition("foo", 1), asList(1, 2, 3))
    +                new SimpleImmutableEntry<>(new TopicPartition("foo", 0), asList(0, 1, 3)),
    +                new SimpleImmutableEntry<>(new TopicPartition("foo", 1), asList(1, 2, 3))
                 ));
     
    -            assertEquals(expStates, actual.v1);
    -            assertFalse(actual.v2);
    +            assertEquals(expStates, actual.getKey());
    +            assertFalse(actual.getValue());
             }
         }
     
    @@ -338,13 +339,13 @@ public void testParseGenerateAssignmentArgs() throws Exception {
                 assertThrows(AdminCommandFailedException.class, () -> parseGenerateAssignmentArgs(
                     "{\"topics\": [{\"topic\": \"foo\"}], \"version\":1}", "5,2,3,4,5"),
                     "Expected to detect duplicate broker list entries").getMessage());
    -        assertEquals(new Tuple2<>(asList(5, 2, 3, 4), asList("foo")),
    +        assertEquals(new SimpleImmutableEntry<>(asList(5, 2, 3, 4), asList("foo")),
                 parseGenerateAssignmentArgs("{\"topics\": [{\"topic\": \"foo\"}], \"version\":1}", "5,2,3,4"));
             assertStartsWith("List of topics to reassign contains duplicate entries",
                 assertThrows(AdminCommandFailedException.class, () -> parseGenerateAssignmentArgs(
                     "{\"topics\": [{\"topic\": \"foo\"},{\"topic\": \"foo\"}], \"version\":1}", "5,2,3,4"),
                     "Expected to detect duplicate topic entries").getMessage());
    -        assertEquals(new Tuple2<>(asList(5, 3, 4), asList("foo", "bar")),
    +        assertEquals(new SimpleImmutableEntry<>(asList(5, 3, 4), asList("foo", "bar")),
                 parseGenerateAssignmentArgs(
                     "{\"topics\": [{\"topic\": \"foo\"},{\"topic\": \"bar\"}], \"version\":1}", "5,3,4"));
         }
    @@ -389,7 +390,7 @@ public void testGenerateAssignmentWithInconsistentRacks() throws Exception {
                         () -> generateAssignment(adminClient, "{\"topics\":[{\"topic\":\"foo\"}]}", "0,1,2,3", true),
                         "Expected generateAssignment to fail").getMessage());
                 // It should succeed when --disable-rack-aware is used.
    -            Tuple2>, Map>>
    +            Entry>, Map>>
                     proposedCurrent = generateAssignment(adminClient, "{\"topics\":[{\"topic\":\"foo\"}]}", "0,1,2,3", false);
     
                 Map> expCurrent = new HashMap<>();
    @@ -397,7 +398,7 @@ public void testGenerateAssignmentWithInconsistentRacks() throws Exception {
                 expCurrent.put(new TopicPartition("foo", 0), asList(0, 1, 2));
                 expCurrent.put(new TopicPartition("foo", 1), asList(1, 2, 3));
     
    -            assertEquals(expCurrent, proposedCurrent.v2);
    +            assertEquals(expCurrent, proposedCurrent.getValue());
             }
         }
     
    @@ -407,7 +408,7 @@ public void testGenerateAssignmentWithFewerBrokers() throws Exception {
                 addTopics(adminClient);
                 List goalBrokers = asList(0, 1, 3);
     
    -            Tuple2>, Map>>
    +            Entry>, Map>>
                     proposedCurrent = generateAssignment(adminClient,
                         "{\"topics\":[{\"topic\":\"foo\"},{\"topic\":\"bar\"}]}",
                         goalBrokers.stream().map(Object::toString).collect(Collectors.joining(",")), false);
    @@ -418,12 +419,12 @@ public void testGenerateAssignmentWithFewerBrokers() throws Exception {
                 expCurrent.put(new TopicPartition("foo", 1), asList(1, 2, 3));
                 expCurrent.put(new TopicPartition("bar", 0), asList(2, 3, 0));
     
    -            assertEquals(expCurrent, proposedCurrent.v2);
    +            assertEquals(expCurrent, proposedCurrent.getValue());
     
                 // The proposed assignment should only span the provided brokers
    -            proposedCurrent.v1.values().forEach(replicas ->
    +            proposedCurrent.getKey().values().forEach(replicas ->
                     assertTrue(goalBrokers.containsAll(replicas),
    -                    "Proposed assignment " + proposedCurrent.v1 + " puts replicas on brokers other than " + goalBrokers)
    +                    "Proposed assignment " + proposedCurrent.getKey() + " puts replicas on brokers other than " + goalBrokers)
                 );
             }
         }
    @@ -567,14 +568,14 @@ public void testParseExecuteAssignmentArgs() throws Exception {
             partitionsToBeReassigned.put(new TopicPartition("foo", 0), asList(1, 2, 3));
             partitionsToBeReassigned.put(new TopicPartition("foo", 1), asList(3, 4, 5));
     
    -        Tuple2>, Map> actual = parseExecuteAssignmentArgs(
    +        Entry>, Map> actual = parseExecuteAssignmentArgs(
                 "{\"version\":1,\"partitions\":" +
                     "[{\"topic\":\"foo\",\"partition\":0,\"replicas\":[1,2,3],\"log_dirs\":[\"any\",\"any\",\"any\"]}," +
                     "{\"topic\":\"foo\",\"partition\":1,\"replicas\":[3,4,5],\"log_dirs\":[\"any\",\"any\",\"any\"]}" +
                     "]}");
     
    -        assertEquals(partitionsToBeReassigned, actual.v1);
    -        assertTrue(actual.v2.isEmpty());
    +        assertEquals(partitionsToBeReassigned, actual.getKey());
    +        assertTrue(actual.getValue().isEmpty());
     
             Map replicaAssignment = new HashMap<>();
     
    @@ -587,8 +588,8 @@ public void testParseExecuteAssignmentArgs() throws Exception {
                     "[{\"topic\":\"foo\",\"partition\":0,\"replicas\":[1,2,3],\"log_dirs\":[\"/tmp/a\",\"/tmp/b\",\"/tmp/c\"]}" +
                     "]}");
     
    -        assertEquals(Collections.singletonMap(new TopicPartition("foo", 0), asList(1, 2, 3)), actual.v1);
    -        assertEquals(replicaAssignment, actual.v2);
    +        assertEquals(Collections.singletonMap(new TopicPartition("foo", 0), asList(1, 2, 3)), actual.getKey());
    +        assertEquals(replicaAssignment, actual.getValue());
         }
     
         @Test
    diff --git a/tools/src/test/resources/log4j.properties b/tools/src/test/resources/log4j.properties
    index 5291604d49..3aca07dc53 100644
    --- a/tools/src/test/resources/log4j.properties
    +++ b/tools/src/test/resources/log4j.properties
    @@ -12,7 +12,7 @@
     # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
     # See the License for the specific language governing permissions and
     # limitations under the License.
    -log4j.rootLogger=TRACE, stdout
    +log4j.rootLogger=INFO, stdout
     
     log4j.appender.stdout=org.apache.log4j.ConsoleAppender
     log4j.appender.stdout.layout=org.apache.log4j.PatternLayout