diff --git a/.gitignore b/.gitignore index 3080a0d60b1d8..dcb1531b4986f 100644 --- a/.gitignore +++ b/.gitignore @@ -53,6 +53,6 @@ systest/ *.swp jmh-benchmarks/generated jmh-benchmarks/src/main/generated -raft/.jqwik-database +**/.jqwik-database **/src/generated **/src/generated-test diff --git a/Jenkinsfile b/Jenkinsfile index 7946f9a7fbeee..985c19064bd2b 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -29,15 +29,10 @@ def isChangeRequest(env) { env.CHANGE_ID != null && !env.CHANGE_ID.isEmpty() } -def retryFlagsString(env) { - if (isChangeRequest(env)) " -PmaxTestRetries=1 -PmaxTestRetryFailures=5" - else "" -} - def doTest(env, target = "unitTest integrationTest") { sh """./gradlew -PscalaVersion=$SCALA_VERSION ${target} \ --profile --no-daemon --continue -PtestLoggingEvents=started,passed,skipped,failed \ - -PignoreFailures=true -PmaxParallelForks=2""" + retryFlagsString(env) + -PignoreFailures=true -PmaxParallelForks=2 -PmaxTestRetries=1 -PmaxTestRetryFailures=10""" junit '**/build/test-results/**/TEST-*.xml' } @@ -160,72 +155,6 @@ pipeline { echo 'Skipping Kafka Streams archetype test for Java 17' } } - - stage('ARM') { - options { - timestamps() - } - environment { - SCALA_VERSION=2.12 - } - stages { - stage('Check ARM Agent') { - agent { label 'arm4' } - options { - timeout(time: 5, unit: 'MINUTES') - } - steps { - echo 'ARM ok' - } - } - stage('Run ARM Build') { - agent { label 'arm4' } - options { - timeout(time: 2, unit: 'HOURS') - } - steps { - doValidation() - catchError(buildResult: 'SUCCESS', stageResult: 'FAILURE') { - doTest(env, 'unitTest') - } - echo 'Skipping Kafka Streams archetype test for ARM build' - } - } - } - } - - stage('PowerPC') { - options { - timestamps() - } - environment { - SCALA_VERSION=2.12 - } - stages { - stage('Check PowerPC Agent') { - agent { label 'ppc64le' } - options { - timeout(time: 5, unit: 'MINUTES') - } - steps { - echo 'PowerPC ok' - } - } - stage('Run PowerPC Build') { - agent { label 'ppc64le' } - options { - timeout(time: 2, unit: 'HOURS') - } - steps { - doValidation() - catchError(buildResult: 'SUCCESS', stageResult: 'FAILURE') { - doTest(env, 'unitTest') - } - echo 'Skipping Kafka Streams archetype test for PowerPC build' - } - } - } - } // To avoid excessive Jenkins resource usage, we only run the stages // above at the PR stage. The ones below are executed after changes diff --git a/LICENSE-binary b/LICENSE-binary index c2400694f2525..a3f860e62b70d 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -207,29 +207,31 @@ License Version 2.0: audience-annotations-0.5.0 commons-cli-1.4 -commons-lang3-3.8.1 -jackson-annotations-2.12.6 -jackson-core-2.12.6 -jackson-databind-2.12.6.1 -jackson-dataformat-csv-2.12.6 -jackson-datatype-jdk8-2.12.6 -jackson-jaxrs-base-2.12.6 -jackson-jaxrs-json-provider-2.12.6 -jackson-module-jaxb-annotations-2.12.6 -jackson-module-paranamer-2.10.5 -jackson-module-scala_2.13-2.12.6 +commons-lang3-3.12.0 +jackson-annotations-2.13.4 +jackson-core-2.13.4 +jackson-databind-2.13.4.2 +jackson-dataformat-csv-2.13.4 +jackson-dataformat-yaml-2.13.4 +jackson-datatype-jdk8-2.13.4 +jackson-datatype-jsr310-2.13.4 +jackson-jaxrs-base-2.13.4 +jackson-jaxrs-json-provider-2.13.4 +jackson-module-jaxb-annotations-2.13.4 +jackson-module-scala_2.13-2.13.4 +jackson-module-scala_2.12-2.13.4 jakarta.validation-api-2.0.2 javassist-3.27.0-GA -jetty-client-9.4.44.v20210927 -jetty-continuation-9.4.44.v20210927 -jetty-http-9.4.44.v20210927 -jetty-io-9.4.44.v20210927 -jetty-security-9.4.44.v20210927 -jetty-server-9.4.44.v20210927 -jetty-servlet-9.4.44.v20210927 -jetty-servlets-9.4.44.v20210927 -jetty-util-9.4.44.v20210927 -jetty-util-ajax-9.4.44.v20210927 +jetty-client-9.4.48.v20220622 +jetty-continuation-9.4.48.v20220622 +jetty-http-9.4.48.v20220622 +jetty-io-9.4.48.v20220622 +jetty-security-9.4.48.v20220622 +jetty-server-9.4.48.v20220622 +jetty-servlet-9.4.48.v20220622 +jetty-servlets-9.4.48.v20220622 +jetty-util-9.4.48.v20220622 +jetty-util-ajax-9.4.48.v20220622 jersey-common-2.34 jersey-server-2.34 jose4j-0.7.9 @@ -237,16 +239,15 @@ lz4-java-1.8.0 maven-artifact-3.8.4 metrics-core-4.1.12.1 metrics-core-2.2.0 -netty-buffer-4.1.73.Final -netty-codec-4.1.73.Final -netty-common-4.1.73.Final -netty-handler-4.1.73.Final -netty-resolver-4.1.73.Final -netty-tcnative-classes-2.0.46.Final -netty-transport-4.1.73.Final -netty-transport-classes-epoll-4.1.73.Final -netty-transport-native-epoll-4.1.73.Final -netty-transport-native-unix-common-4.1.73.Final +netty-buffer-4.1.78.Final +netty-codec-4.1.78.Final +netty-common-4.1.78.Final +netty-handler-4.1.78.Final +netty-resolver-4.1.78.Final +netty-transport-4.1.78.Final +netty-transport-classes-epoll-4.1.78.Final +netty-transport-native-epoll-4.1.78.Final +netty-transport-native-unix-common-4.1.78.Final plexus-utils-3.3.0 reload4j-1.2.19 rocksdbjni-6.29.4.1 @@ -255,7 +256,13 @@ scala-library-2.13.8 scala-logging_2.13-3.9.4 scala-reflect-2.13.8 scala-java8-compat_2.13-1.0.2 +snakeyaml-1.30 snappy-java-1.1.8.4 +swagger-annotations-2.2.0 +swagger-core-2.2.0 +swagger-integration-2.2.0 +swagger-jaxrs2-2.2.0 +swagger-models-2.2.0 zookeeper-3.6.3 zookeeper-jute-3.6.3 @@ -268,8 +275,8 @@ See licenses/ for text of these licenses. Eclipse Distribution License - v 1.0 see: licenses/eclipse-distribution-license-1.0 -jakarta.activation-api-1.2.1 -jakarta.xml.bind-api-2.3.2 +jakarta.activation-api-1.2.2 +jakarta.xml.bind-api-2.3.3 --------------------------------------- Eclipse Public License - v 2.0 @@ -305,6 +312,7 @@ argparse4j-0.7.0, see: licenses/argparse-MIT jopt-simple-5.0.4, see: licenses/jopt-simple-MIT slf4j-api-1.7.36, see: licenses/slf4j-MIT slf4j-reload4j-1.7.36, see: licenses/slf4j-MIT +classgraph-4.8.138, see: license/classgraph-MIT --------------------------------------- BSD 2-Clause diff --git a/README.md b/README.md index 2aa509e081012..36f71d122c8aa 100644 --- a/README.md +++ b/README.md @@ -83,15 +83,17 @@ fail due to code changes. You can just run: ./gradlew processMessages processTestMessages +### Running a Kafka broker in KRaft mode + + KAFKA_CLUSTER_ID="$(./bin/kafka-storage.sh random-uuid)" + ./bin/kafka-storage.sh format -t $KAFKA_CLUSTER_ID -c config/kraft/server.properties + ./bin/kafka-server-start.sh config/kraft/server.properties + ### Running a Kafka broker in ZooKeeper mode ./bin/zookeeper-server-start.sh config/zookeeper.properties ./bin/kafka-server-start.sh config/server.properties -### Running a Kafka broker in KRaft (Kafka Raft metadata) mode - -See [config/kraft/README.md](https://github.com/apache/kafka/blob/trunk/config/kraft/README.md). - ### Cleaning the build ### ./gradlew clean diff --git a/bin/kafka-metadata-quorum.sh b/bin/kafka-metadata-quorum.sh new file mode 100755 index 0000000000000..24bedbded1e7d --- /dev/null +++ b/bin/kafka-metadata-quorum.sh @@ -0,0 +1,17 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +exec $(dirname $0)/kafka-run-class.sh kafka.admin.MetadataQuorumCommand "$@" diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 490f930b8cc4d..97ef0f5e8f089 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -213,6 +213,10 @@ fi # JMX port to use if [ $JMX_PORT ]; then KAFKA_JMX_OPTS="$KAFKA_JMX_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT " + if ! echo "$KAFKA_JMX_OPTS" | grep -qF -- '-Dcom.sun.management.jmxremote.rmi.port=' ; then + # If unset, set the RMI port to address issues with monitoring Kafka running in containers + KAFKA_JMX_OPTS="$KAFKA_JMX_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT" + fi fi # Log directory to use diff --git a/bin/windows/kafka-metatada-quorum.bat b/bin/windows/kafka-metatada-quorum.bat new file mode 100644 index 0000000000000..4ea8e3109f962 --- /dev/null +++ b/bin/windows/kafka-metatada-quorum.bat @@ -0,0 +1,17 @@ +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +"%~dp0kafka-run-class.bat" kafka.admin.MetadataQuorumCommand %* diff --git a/build.gradle b/build.gradle index 73c36c31170fa..56511cf61ce63 100644 --- a/build.gradle +++ b/build.gradle @@ -875,6 +875,7 @@ project(':core') { testImplementation project(':clients').sourceSets.test.output testImplementation project(':metadata').sourceSets.test.output testImplementation project(':raft').sourceSets.test.output + testImplementation project(':server-common').sourceSets.test.output testImplementation libs.bcpkix testImplementation libs.mockitoCore testImplementation(libs.apacheda) { @@ -1151,12 +1152,14 @@ project(':metadata') { implementation libs.metrics compileOnly libs.log4j testImplementation libs.junitJupiter + testImplementation libs.jqwik testImplementation libs.hamcrest testImplementation libs.mockitoCore testImplementation libs.mockitoInline testImplementation libs.slf4jlog4j testImplementation project(':clients').sourceSets.test.output testImplementation project(':raft').sourceSets.test.output + testImplementation project(':server-common').sourceSets.test.output generator project(':generator') } @@ -1595,6 +1598,8 @@ project(':storage') { testImplementation project(':clients').sourceSets.test.output testImplementation project(':core') testImplementation project(':core').sourceSets.test.output + testImplementation project(':server-common') + testImplementation project(':server-common').sourceSets.test.output testImplementation libs.junitJupiter testImplementation libs.mockitoCore testImplementation libs.bcpkix @@ -1679,6 +1684,7 @@ project(':tools') { dependencies { implementation project(':clients') + implementation project(':server-common') implementation project(':log4j-appender') implementation libs.argparse4j implementation libs.jacksonDatabind @@ -2534,9 +2540,10 @@ project(':connect:runtime') { implementation libs.jettyClient implementation libs.reflections implementation libs.mavenArtifact - implementation libs.swaggerJaxrs2 implementation libs.swaggerAnnotations + compileOnly libs.swaggerJaxrs2 + testImplementation project(':clients').sourceSets.test.output testImplementation project(':core') testImplementation project(':metadata') @@ -2624,7 +2631,8 @@ project(':connect:runtime') { } task genConnectOpenAPIDocs(type: io.swagger.v3.plugins.gradle.tasks.ResolveTask, dependsOn: setVersionInOpenAPISpec) { - classpath = sourceSets.main.runtimeClasspath + classpath = sourceSets.main.compileClasspath + sourceSets.main.runtimeClasspath + buildClasspath = classpath outputFileName = 'connect_rest' outputFormat = 'YAML' diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index d0599f3d7a34d..bf2d339da8c76 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -143,9 +143,15 @@ + + + + + + diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml index 28b325b093dae..4042cba402fdd 100644 --- a/checkstyle/import-control-core.xml +++ b/checkstyle/import-control-core.xml @@ -54,6 +54,7 @@ + diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 211d23ff60aeb..44260f360bbf3 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -36,6 +36,7 @@ + @@ -232,6 +233,7 @@ + @@ -275,6 +277,13 @@ + + + + + + + @@ -362,6 +371,7 @@ + diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index f6ca0d02fe316..7c32223961b56 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -39,6 +39,8 @@ + + + + + @@ -276,6 +284,10 @@ + + + files="(QuorumController|QuorumControllerTest|ReplicationControlManager|ReplicationControlManagerTest|ClusterControlManagerTest).java"/> sessionTopicPartitions() { + return sessionPartitions.keySet(); + } } 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 fdacc09db8c8a..1d469a66436a6 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 @@ -36,6 +36,7 @@ import java.time.Duration; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -919,12 +920,21 @@ default ListConsumerGroupsResult listConsumerGroups() { * @param options The options to use when listing the consumer group offsets. * @return The ListGroupOffsetsResult */ - ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, ListConsumerGroupOffsetsOptions options); + default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, ListConsumerGroupOffsetsOptions options) { + @SuppressWarnings("deprecation") + ListConsumerGroupOffsetsSpec groupSpec = new ListConsumerGroupOffsetsSpec() + .topicPartitions(options.topicPartitions()); + + // We can use the provided options with the batched API, which uses topic partitions from + // the group spec and ignores any topic partitions set in the options. + return listConsumerGroupOffsets(Collections.singletonMap(groupId, groupSpec), options); + } /** * List the consumer group offsets available in the cluster with the default options. *

- * This is a convenience method for {@link #listConsumerGroupOffsets(String, ListConsumerGroupOffsetsOptions)} with default options. + * This is a convenience method for {@link #listConsumerGroupOffsets(Map, ListConsumerGroupOffsetsOptions)} + * to list offsets of all partitions of one group with default options. * * @return The ListGroupOffsetsResult. */ @@ -932,6 +942,29 @@ default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId) return listConsumerGroupOffsets(groupId, new ListConsumerGroupOffsetsOptions()); } + /** + * List the consumer group offsets available in the cluster for the specified consumer groups. + * + * @param groupSpecs Map of consumer group ids to a spec that specifies the topic partitions of the group to list offsets for. + * + * @param options The options to use when listing the consumer group offsets. + * @return The ListConsumerGroupOffsetsResult + */ + ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map groupSpecs, ListConsumerGroupOffsetsOptions options); + + /** + * List the consumer group offsets available in the cluster for the specified groups with the default options. + *

+ * This is a convenience method for + * {@link #listConsumerGroupOffsets(Map, ListConsumerGroupOffsetsOptions)} with default options. + * + * @param groupSpecs Map of consumer group ids to a spec that specifies the topic partitions of the group to list offsets for. + * @return The ListConsumerGroupOffsetsResult. + */ + default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map groupSpecs) { + return listConsumerGroupOffsets(groupSpecs, new ListConsumerGroupOffsetsOptions()); + } + /** * Delete consumer groups from the cluster. * diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java b/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java index aa0401a8a86eb..1442de58513f9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java @@ -36,10 +36,10 @@ public class FinalizedVersionRange { * @throws IllegalArgumentException Raised when the condition described above is not met. */ FinalizedVersionRange(final short minVersionLevel, final short maxVersionLevel) { - if (minVersionLevel < 1 || maxVersionLevel < 1 || maxVersionLevel < minVersionLevel) { + if (minVersionLevel < 0 || maxVersionLevel < 0 || maxVersionLevel < minVersionLevel) { throw new IllegalArgumentException( String.format( - "Expected minVersionLevel >= 1, maxVersionLevel >= 1 and" + + "Expected minVersionLevel >= 0, maxVersionLevel >= 0 and" + " maxVersionLevel >= minVersionLevel, but received" + " minVersionLevel: %d, maxVersionLevel: %d", minVersionLevel, maxVersionLevel)); } 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 aad2610c94a45..89bc011a4d504 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 @@ -218,6 +218,7 @@ import org.apache.kafka.common.requests.ExpireDelegationTokenResponse; import org.apache.kafka.common.requests.IncrementalAlterConfigsRequest; import org.apache.kafka.common.requests.IncrementalAlterConfigsResponse; +import org.apache.kafka.common.requests.JoinGroupRequest; import org.apache.kafka.common.requests.ListGroupsRequest; import org.apache.kafka.common.requests.ListGroupsResponse; import org.apache.kafka.common.requests.ListOffsetsRequest; @@ -273,8 +274,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.kafka.common.internals.Topic.METADATA_TOPIC_NAME; -import static org.apache.kafka.common.internals.Topic.METADATA_TOPIC_PARTITION; +import static org.apache.kafka.common.internals.Topic.CLUSTER_METADATA_TOPIC_NAME; +import static org.apache.kafka.common.internals.Topic.CLUSTER_METADATA_TOPIC_PARTITION; import static org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData.ReassignablePartition; import static org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignablePartitionResponse; import static org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignableTopicResponse; @@ -3400,13 +3401,14 @@ void handleFailure(Throwable throwable) { } @Override - public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(final String groupId, - final ListConsumerGroupOffsetsOptions options) { + public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map groupSpecs, + ListConsumerGroupOffsetsOptions options) { SimpleAdminApiFuture> future = - ListConsumerGroupOffsetsHandler.newFuture(groupId); - ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, options.topicPartitions(), options.requireStable(), logContext); + ListConsumerGroupOffsetsHandler.newFuture(groupSpecs.keySet()); + ListConsumerGroupOffsetsHandler handler = + new ListConsumerGroupOffsetsHandler(groupSpecs, options.requireStable(), logContext); invokeDriver(handler, future, options.timeoutMs); - return new ListConsumerGroupOffsetsResult(future.get(CoordinatorKey.byGroupId(groupId))); + return new ListConsumerGroupOffsetsResult(future.all()); } @Override @@ -3756,7 +3758,7 @@ private List getMembersFromGroup(String groupId, String reason) public RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroup(String groupId, RemoveMembersFromConsumerGroupOptions options) { String reason = options.reason() == null || options.reason().isEmpty() ? - DEFAULT_LEAVE_GROUP_REASON : options.reason(); + DEFAULT_LEAVE_GROUP_REASON : JoinGroupRequest.maybeTruncateReason(options.reason()); List members; if (options.removeAll()) { @@ -4355,16 +4357,27 @@ private QuorumInfo.ReplicaState translateReplicaState(DescribeQuorumResponseData } private QuorumInfo createQuorumResult(final DescribeQuorumResponseData.PartitionData partition) { + List voters = partition.currentVoters().stream() + .map(this::translateReplicaState) + .collect(Collectors.toList()); + + List observers = partition.observers().stream() + .map(this::translateReplicaState) + .collect(Collectors.toList()); + return new QuorumInfo( - partition.leaderId(), - partition.currentVoters().stream().map(v -> translateReplicaState(v)).collect(Collectors.toList()), - partition.observers().stream().map(o -> translateReplicaState(o)).collect(Collectors.toList())); + partition.leaderId(), + partition.leaderEpoch(), + partition.highWatermark(), + voters, + observers + ); } @Override DescribeQuorumRequest.Builder createRequest(int timeoutMs) { return new Builder(DescribeQuorumRequest.singletonRequest( - new TopicPartition(METADATA_TOPIC_NAME, METADATA_TOPIC_PARTITION.partition()))); + new TopicPartition(CLUSTER_METADATA_TOPIC_NAME, CLUSTER_METADATA_TOPIC_PARTITION.partition()))); } @Override @@ -4380,9 +4393,9 @@ void handleResponse(AbstractResponse response) { throw new UnknownServerException(msg); } DescribeQuorumResponseData.TopicData topic = quorumResponse.data().topics().get(0); - if (!topic.topicName().equals(METADATA_TOPIC_NAME)) { + if (!topic.topicName().equals(CLUSTER_METADATA_TOPIC_NAME)) { String msg = String.format("DescribeMetadataQuorum received a topic with name %s when %s was expected", - topic.topicName(), METADATA_TOPIC_NAME); + topic.topicName(), CLUSTER_METADATA_TOPIC_NAME); log.debug(msg); throw new UnknownServerException(msg); } @@ -4393,9 +4406,9 @@ void handleResponse(AbstractResponse response) { throw new UnknownServerException(msg); } DescribeQuorumResponseData.PartitionData partition = topic.partitions().get(0); - if (partition.partitionIndex() != METADATA_TOPIC_PARTITION.partition()) { + if (partition.partitionIndex() != CLUSTER_METADATA_TOPIC_PARTITION.partition()) { String msg = String.format("DescribeMetadataQuorum received a single partition with index %d when %d was expected", - partition.partitionIndex(), METADATA_TOPIC_PARTITION.partition()); + partition.partitionIndex(), CLUSTER_METADATA_TOPIC_PARTITION.partition()); log.debug(msg); throw new UnknownServerException(msg); } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsOptions.java index 292a47ef393c3..44d3a407327e1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsOptions.java @@ -23,23 +23,28 @@ import java.util.List; /** - * Options for {@link Admin#listConsumerGroupOffsets(String)}. + * Options for {@link Admin#listConsumerGroupOffsets(java.util.Map)} and {@link Admin#listConsumerGroupOffsets(String)}. *

* The API of this class is evolving, see {@link Admin} for details. */ @InterfaceStability.Evolving public class ListConsumerGroupOffsetsOptions extends AbstractOptions { - private List topicPartitions = null; + private List topicPartitions; private boolean requireStable = false; /** * Set the topic partitions to list as part of the result. * {@code null} includes all topic partitions. + *

+ * @deprecated Since 3.3. + * Use {@link Admin#listConsumerGroupOffsets(java.util.Map, ListConsumerGroupOffsetsOptions)} + * to specify topic partitions. * * @param topicPartitions List of topic partitions to include * @return This ListGroupOffsetsOptions */ + @Deprecated public ListConsumerGroupOffsetsOptions topicPartitions(List topicPartitions) { this.topicPartitions = topicPartitions; return this; @@ -55,7 +60,12 @@ public ListConsumerGroupOffsetsOptions requireStable(final boolean requireStable /** * Returns a list of topic partitions to add as part of the result. + *

+ * @deprecated Since 3.3. + * Use {@link Admin#listConsumerGroupOffsets(java.util.Map, ListConsumerGroupOffsetsOptions)} + * to specify topic partitions. */ + @Deprecated public List topicPartitions() { return topicPartitions; } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsResult.java index 48f4531418110..2136e33a401e1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsResult.java @@ -17,25 +17,32 @@ package org.apache.kafka.clients.admin; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +import org.apache.kafka.clients.admin.internals.CoordinatorKey; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.annotation.InterfaceStability; -import java.util.Map; - /** - * The result of the {@link Admin#listConsumerGroupOffsets(String)} call. + * The result of the {@link Admin#listConsumerGroupOffsets(Map)} and + * {@link Admin#listConsumerGroupOffsets(String)} call. *

* The API of this class is evolving, see {@link Admin} for details. */ @InterfaceStability.Evolving public class ListConsumerGroupOffsetsResult { - final KafkaFuture> future; + final Map>> futures; - ListConsumerGroupOffsetsResult(KafkaFuture> future) { - this.future = future; + ListConsumerGroupOffsetsResult(final Map>> futures) { + this.futures = futures.entrySet().stream() + .collect(Collectors.toMap(e -> e.getKey().idValue, Entry::getValue)); } /** @@ -43,7 +50,42 @@ public class ListConsumerGroupOffsetsResult { * If the group does not have a committed offset for this partition, the corresponding value in the returned map will be null. */ public KafkaFuture> partitionsToOffsetAndMetadata() { - return future; + if (futures.size() != 1) { + throw new IllegalStateException("Offsets from multiple consumer groups were requested. " + + "Use partitionsToOffsetAndMetadata(groupId) instead to get future for a specific group."); + } + return futures.values().iterator().next(); } + /** + * Return a future which yields a map of topic partitions to OffsetAndMetadata objects for + * the specified group. If the group doesn't have a committed offset for a specific + * partition, the corresponding value in the returned map will be null. + */ + public KafkaFuture> partitionsToOffsetAndMetadata(String groupId) { + if (!futures.containsKey(groupId)) + throw new IllegalArgumentException("Offsets for consumer group '" + groupId + "' were not requested."); + return futures.get(groupId); + } + + /** + * Return a future which yields all Map objects, + * if requests for all the groups succeed. + */ + public KafkaFuture>> all() { + return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])).thenApply( + nil -> { + Map> listedConsumerGroupOffsets = new HashMap<>(futures.size()); + futures.forEach((key, future) -> { + try { + listedConsumerGroupOffsets.put(key, future.get()); + } catch (InterruptedException | ExecutionException e) { + // This should be unreachable, since the KafkaFuture#allOf already ensured + // that all of the futures completed successfully. + throw new RuntimeException(e); + } + }); + return listedConsumerGroupOffsets; + }); + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsSpec.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsSpec.java new file mode 100644 index 0000000000000..83858e49c8170 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListConsumerGroupOffsetsSpec.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Collection; +import java.util.Objects; + +/** + * Specification of consumer group offsets to list using {@link Admin#listConsumerGroupOffsets(java.util.Map)}. + * + * The API of this class is evolving, see {@link Admin} for details. + */ +@InterfaceStability.Evolving +public class ListConsumerGroupOffsetsSpec { + + private Collection topicPartitions; + + /** + * Set the topic partitions whose offsets are to be listed for a consumer group. + * {@code null} includes all topic partitions. + * + * @param topicPartitions List of topic partitions to include + * @return This ListConsumerGroupOffsetSpec + */ + public ListConsumerGroupOffsetsSpec topicPartitions(Collection topicPartitions) { + this.topicPartitions = topicPartitions; + return this; + } + + /** + * Returns the topic partitions whose offsets are to be listed for a consumer group. + * {@code null} indicates that offsets of all partitions of the group are to be listed. + */ + public Collection topicPartitions() { + return topicPartitions; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof ListConsumerGroupOffsetsSpec)) { + return false; + } + ListConsumerGroupOffsetsSpec that = (ListConsumerGroupOffsetsSpec) o; + return Objects.equals(topicPartitions, that.topicPartitions); + } + + @Override + public int hashCode() { + return Objects.hash(topicPartitions); + } + + @Override + public String toString() { + return "ListConsumerGroupOffsetsSpec(" + + "topicPartitions=" + topicPartitions + + ')'; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java b/clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java index 75476d77dcff1..f9e4f8c11c904 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java @@ -24,20 +24,38 @@ * This class is used to describe the state of the quorum received in DescribeQuorumResponse. */ public class QuorumInfo { - private final Integer leaderId; + private final int leaderId; + private final long leaderEpoch; + private final long highWatermark; private final List voters; private final List observers; - QuorumInfo(Integer leaderId, List voters, List observers) { + QuorumInfo( + int leaderId, + long leaderEpoch, + long highWatermark, + List voters, + List observers + ) { this.leaderId = leaderId; + this.leaderEpoch = leaderEpoch; + this.highWatermark = highWatermark; this.voters = voters; this.observers = observers; } - public Integer leaderId() { + public int leaderId() { return leaderId; } + public long leaderEpoch() { + return leaderEpoch; + } + + public long highWatermark() { + return highWatermark; + } + public List voters() { return voters; } @@ -51,20 +69,24 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; QuorumInfo that = (QuorumInfo) o; - return leaderId.equals(that.leaderId) - && voters.equals(that.voters) - && observers.equals(that.observers); + return leaderId == that.leaderId + && leaderEpoch == that.leaderEpoch + && highWatermark == that.highWatermark + && Objects.equals(voters, that.voters) + && Objects.equals(observers, that.observers); } @Override public int hashCode() { - return Objects.hash(leaderId, voters, observers); + return Objects.hash(leaderId, leaderEpoch, highWatermark, voters, observers); } @Override public String toString() { return "QuorumInfo(" + "leaderId=" + leaderId + + ", leaderEpoch=" + leaderEpoch + + ", highWatermark=" + highWatermark + ", voters=" + voters + ", observers=" + observers + ')'; @@ -73,8 +95,8 @@ public String toString() { public static class ReplicaState { private final int replicaId; private final long logEndOffset; - private final OptionalLong lastFetchTimeMs; - private final OptionalLong lastCaughtUpTimeMs; + private final OptionalLong lastFetchTimestamp; + private final OptionalLong lastCaughtUpTimestamp; ReplicaState() { this(0, 0, OptionalLong.empty(), OptionalLong.empty()); @@ -83,13 +105,13 @@ public static class ReplicaState { ReplicaState( int replicaId, long logEndOffset, - OptionalLong lastFetchTimeMs, - OptionalLong lastCaughtUpTimeMs + OptionalLong lastFetchTimestamp, + OptionalLong lastCaughtUpTimestamp ) { this.replicaId = replicaId; this.logEndOffset = logEndOffset; - this.lastFetchTimeMs = lastFetchTimeMs; - this.lastCaughtUpTimeMs = lastCaughtUpTimeMs; + this.lastFetchTimestamp = lastFetchTimestamp; + this.lastCaughtUpTimestamp = lastCaughtUpTimestamp; } /** @@ -109,19 +131,21 @@ public long logEndOffset() { } /** - * Return the lastFetchTime in milliseconds for this replica. + * Return the last millisecond timestamp that the leader received a + * fetch from this replica. * @return The value of the lastFetchTime if known, empty otherwise */ - public OptionalLong lastFetchTimeMs() { - return lastFetchTimeMs; + public OptionalLong lastFetchTimestamp() { + return lastFetchTimestamp; } /** - * Return the lastCaughtUpTime in milliseconds for this replica. + * Return the last millisecond timestamp at which this replica was known to be + * caught up with the leader. * @return The value of the lastCaughtUpTime if known, empty otherwise */ - public OptionalLong lastCaughtUpTimeMs() { - return lastCaughtUpTimeMs; + public OptionalLong lastCaughtUpTimestamp() { + return lastCaughtUpTimestamp; } @Override @@ -131,13 +155,13 @@ public boolean equals(Object o) { ReplicaState that = (ReplicaState) o; return replicaId == that.replicaId && logEndOffset == that.logEndOffset - && lastFetchTimeMs.equals(that.lastFetchTimeMs) - && lastCaughtUpTimeMs.equals(that.lastCaughtUpTimeMs); + && lastFetchTimestamp.equals(that.lastFetchTimestamp) + && lastCaughtUpTimestamp.equals(that.lastCaughtUpTimestamp); } @Override public int hashCode() { - return Objects.hash(replicaId, logEndOffset, lastFetchTimeMs, lastCaughtUpTimeMs); + return Objects.hash(replicaId, logEndOffset, lastFetchTimestamp, lastCaughtUpTimestamp); } @Override @@ -145,8 +169,8 @@ public String toString() { return "ReplicaState(" + "replicaId=" + replicaId + ", logEndOffset=" + logEndOffset + - ", lastFetchTimeMs=" + lastFetchTimeMs + - ", lastCaughtUpTimeMs=" + lastCaughtUpTimeMs + + ", lastFetchTimestamp=" + lastFetchTimestamp + + ", lastCaughtUpTimestamp=" + lastCaughtUpTimestamp + ')'; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java b/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java index d71da31fb8200..b85a392a658c3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java @@ -36,10 +36,10 @@ public class SupportedVersionRange { * @throws IllegalArgumentException Raised when the condition described above is not met. */ SupportedVersionRange(final short minVersion, final short maxVersion) { - if (minVersion < 1 || maxVersion < 1 || maxVersion < minVersion) { + if (minVersion < 0 || maxVersion < 0 || maxVersion < minVersion) { throw new IllegalArgumentException( String.format( - "Expected 1 <= minVersion <= maxVersion but received minVersion:%d, maxVersion:%d.", + "Expected 0 <= minVersion <= maxVersion but received minVersion:%d, maxVersion:%d.", minVersion, maxVersion)); } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java index d00db4b18c694..0e1b03d964fea 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.FindCoordinatorRequest.NoBatchedFindCoordinatorsException; +import org.apache.kafka.common.requests.OffsetFetchRequest.NoBatchedOffsetFetchRequestException; import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; @@ -253,7 +254,7 @@ public void onFailure( .collect(Collectors.toSet()); retryLookup(keysToUnmap); - } else if (t instanceof NoBatchedFindCoordinatorsException) { + } else if (t instanceof NoBatchedFindCoordinatorsException || t instanceof NoBatchedOffsetFetchRequestException) { ((CoordinatorStrategy) handler.lookupStrategy()).disableBatch(); Set keysToUnmap = spec.keys.stream() .filter(future.lookupKeys()::contains) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java index e6fc0d624a0cd..02b68527c3c7e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java @@ -120,6 +120,10 @@ public void disableBatch() { batch = false; } + public boolean batch() { + return batch; + } + private CoordinatorKey requireSingletonAndType(Set keys) { if (keys.size() != 1) { throw new IllegalArgumentException("Unexpected size of key set: expected 1, but got " + keys.size()); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java index 08648821f7c9f..21c7d8d488f3d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java @@ -17,14 +17,16 @@ package org.apache.kafka.clients.admin.internals; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsSpec; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; @@ -36,39 +38,26 @@ import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; -public class ListConsumerGroupOffsetsHandler extends AdminApiHandler.Batched> { +public class ListConsumerGroupOffsetsHandler implements AdminApiHandler> { - private final CoordinatorKey groupId; - private final List partitions; private final boolean requireStable; + private final Map groupSpecs; private final Logger log; - private final AdminApiLookupStrategy lookupStrategy; + private final CoordinatorStrategy lookupStrategy; public ListConsumerGroupOffsetsHandler( - String groupId, - List partitions, - LogContext logContext - ) { - this(groupId, partitions, false, logContext); - } - - public ListConsumerGroupOffsetsHandler( - String groupId, - List partitions, + Map groupSpecs, boolean requireStable, LogContext logContext ) { - this.groupId = CoordinatorKey.byGroupId(groupId); - this.partitions = partitions; - this.requireStable = requireStable; this.log = logContext.logger(ListConsumerGroupOffsetsHandler.class); this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, logContext); + this.groupSpecs = groupSpecs; + this.requireStable = requireStable; } - public static AdminApiFuture.SimpleAdminApiFuture> newFuture( - String groupId - ) { - return AdminApiFuture.forKeys(Collections.singleton(CoordinatorKey.byGroupId(groupId))); + public static AdminApiFuture.SimpleAdminApiFuture> newFuture(Collection groupIds) { + return AdminApiFuture.forKeys(coordinatorKeys(groupIds)); } @Override @@ -82,16 +71,45 @@ public AdminApiLookupStrategy lookupStrategy() { } private void validateKeys(Set groupIds) { - if (!groupIds.equals(Collections.singleton(groupId))) { + Set keys = coordinatorKeys(groupSpecs.keySet()); + if (!keys.containsAll(groupIds)) { throw new IllegalArgumentException("Received unexpected group ids " + groupIds + - " (expected only " + Collections.singleton(groupId) + ")"); + " (expected one of " + keys + ")"); } } + private static Set coordinatorKeys(Collection groupIds) { + return groupIds.stream() + .map(CoordinatorKey::byGroupId) + .collect(Collectors.toSet()); + } + + public OffsetFetchRequest.Builder buildBatchedRequest(Set groupIds) { + // Create a map that only contains the consumer groups owned by the coordinator. + Map> coordinatorGroupIdToTopicPartitions = new HashMap<>(groupIds.size()); + groupIds.forEach(g -> { + ListConsumerGroupOffsetsSpec spec = groupSpecs.get(g.idValue); + List partitions = spec.topicPartitions() != null ? new ArrayList<>(spec.topicPartitions()) : null; + coordinatorGroupIdToTopicPartitions.put(g.idValue, partitions); + }); + + return new OffsetFetchRequest.Builder(coordinatorGroupIdToTopicPartitions, requireStable, false); + } + @Override - public OffsetFetchRequest.Builder buildBatchedRequest(int coordinatorId, Set groupIds) { + public Collection> buildRequest(int brokerId, Set groupIds) { validateKeys(groupIds); - return new OffsetFetchRequest.Builder(groupId.idValue, requireStable, partitions, false); + + // When the OffsetFetchRequest fails with NoBatchedOffsetFetchRequestException, we completely disable + // the batching end-to-end, including the FindCoordinatorRequest. + if (lookupStrategy.batch()) { + return Collections.singletonList(new RequestAndKeys<>(buildBatchedRequest(groupIds), groupIds)); + } else { + return groupIds.stream().map(groupId -> { + Set keys = Collections.singleton(groupId); + return new RequestAndKeys<>(buildBatchedRequest(keys), keys); + }).collect(Collectors.toList()); + } } @Override @@ -104,44 +122,46 @@ public ApiResult> handleR final OffsetFetchResponse response = (OffsetFetchResponse) abstractResponse; - // the groupError will contain the group level error for v0-v8 OffsetFetchResponse - Errors groupError = response.groupLevelError(groupId.idValue); - if (groupError != Errors.NONE) { - final Map failed = new HashMap<>(); - final Set groupsToUnmap = new HashSet<>(); - - handleGroupError(groupId, groupError, failed, groupsToUnmap); - - return new ApiResult<>(Collections.emptyMap(), failed, new ArrayList<>(groupsToUnmap)); - } else { - final Map groupOffsetsListing = new HashMap<>(); - - response.partitionDataMap(groupId.idValue).forEach((topicPartition, partitionData) -> { - final Errors error = partitionData.error; - if (error == Errors.NONE) { - final long offset = partitionData.offset; - final String metadata = partitionData.metadata; - final Optional leaderEpoch = partitionData.leaderEpoch; - // Negative offset indicates that the group has no committed offset for this partition - if (offset < 0) { - groupOffsetsListing.put(topicPartition, null); + Map> completed = new HashMap<>(); + Map failed = new HashMap<>(); + List unmapped = new ArrayList<>(); + for (CoordinatorKey coordinatorKey : groupIds) { + String group = coordinatorKey.idValue; + if (response.groupHasError(group)) { + handleGroupError(CoordinatorKey.byGroupId(group), response.groupLevelError(group), failed, unmapped); + } else { + final Map groupOffsetsListing = new HashMap<>(); + Map responseData = response.partitionDataMap(group); + for (Map.Entry partitionEntry : responseData.entrySet()) { + final TopicPartition topicPartition = partitionEntry.getKey(); + OffsetFetchResponse.PartitionData partitionData = partitionEntry.getValue(); + final Errors error = partitionData.error; + + if (error == Errors.NONE) { + final long offset = partitionData.offset; + final String metadata = partitionData.metadata; + final Optional leaderEpoch = partitionData.leaderEpoch; + // Negative offset indicates that the group has no committed offset for this partition + if (offset < 0) { + groupOffsetsListing.put(topicPartition, null); + } else { + groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, leaderEpoch, metadata)); + } } else { - groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, leaderEpoch, metadata)); + log.warn("Skipping return offset for {} due to error {}.", topicPartition, error); } - } else { - log.warn("Skipping return offset for {} due to error {}.", topicPartition, error); } - }); - - return ApiResult.completed(groupId, groupOffsetsListing); + completed.put(CoordinatorKey.byGroupId(group), groupOffsetsListing); + } } + return new ApiResult<>(completed, failed, unmapped); } private void handleGroupError( CoordinatorKey groupId, Errors error, Map failed, - Set groupsToUnmap + List groupsToUnmap ) { switch (error) { case GROUP_AUTHORIZATION_FAILED: diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index fa80727d5af43..80b515d601b2a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -187,11 +187,12 @@ public AbstractCoordinator(GroupRebalanceConfig rebalanceConfig, /** * Invoked prior to each group join or rejoin. This is typically used to perform any * cleanup from the previous generation (such as committing offsets for the consumer) + * @param timer Timer bounding how long this method can block * @param generation The previous generation or -1 if there was none * @param memberId The identifier of this member in the previous group or "" if there was none * @return true If onJoinPrepare async commit succeeded, false otherwise */ - protected abstract boolean onJoinPrepare(int generation, String memberId); + protected abstract boolean onJoinPrepare(Timer timer, int generation, String memberId); /** * Invoked when the leader is elected. This is used by the leader to perform the assignment @@ -230,14 +231,27 @@ protected abstract void onJoinComplete(int generation, protected void onLeavePrepare() {} /** - * Visible for testing. - * * Ensure that the coordinator is ready to receive requests. * * @param timer Timer bounding how long this method can block * @return true If coordinator discovery and initial connection succeeded, false otherwise */ protected synchronized boolean ensureCoordinatorReady(final Timer timer) { + return ensureCoordinatorReady(timer, false); + } + + /** + * Ensure that the coordinator is ready to receive requests. This will return + * immediately without blocking. It is intended to be called in an asynchronous + * context when wakeups are not expected. + * + * @return true If coordinator discovery and initial connection succeeded, false otherwise + */ + protected synchronized boolean ensureCoordinatorReadyAsync() { + return ensureCoordinatorReady(time.timer(0), true); + } + + private synchronized boolean ensureCoordinatorReady(final Timer timer, boolean disableWakeup) { if (!coordinatorUnknown()) return true; @@ -248,7 +262,7 @@ protected synchronized boolean ensureCoordinatorReady(final Timer timer) { throw fatalException; } final RequestFuture future = lookupCoordinator(); - client.poll(future, timer); + client.poll(future, timer, disableWakeup); if (!future.isDone()) { // ran out of time @@ -426,7 +440,7 @@ boolean joinGroupIfNeeded(final Timer timer) { // exception, in which case upon retry we should not retry onJoinPrepare either. needsJoinPrepare = false; // return false when onJoinPrepare is waiting for committing offset - if (!onJoinPrepare(generation.generationId, generation.memberId)) { + if (!onJoinPrepare(timer, generation.generationId, generation.memberId)) { needsJoinPrepare = true; //should not initiateJoinGroup if needsJoinPrepare still is true return false; @@ -478,11 +492,11 @@ boolean joinGroupIfNeeded(final Timer timer) { resetJoinGroupFuture(); synchronized (AbstractCoordinator.this) { - final String shortReason = String.format("rebalance failed due to %s", - exception.getClass().getSimpleName()); + final String simpleName = exception.getClass().getSimpleName(); + final String shortReason = String.format("rebalance failed due to %s", simpleName); final String fullReason = String.format("rebalance failed due to '%s' (%s)", exception.getMessage(), - exception.getClass().getSimpleName()); + simpleName); requestRejoin(shortReason, fullReason); } @@ -559,7 +573,7 @@ RequestFuture sendJoinGroupRequest() { .setProtocolType(protocolType()) .setProtocols(metadata()) .setRebalanceTimeoutMs(this.rebalanceConfig.rebalanceTimeoutMs) - .setReason(this.rejoinReason) + .setReason(JoinGroupRequest.maybeTruncateReason(this.rejoinReason)) ); log.debug("Sending JoinGroup ({}) to coordinator {}", requestBuilder, this.coordinator); @@ -811,9 +825,6 @@ public void handle(SyncGroupResponse syncResponse, } else if (error == Errors.REBALANCE_IN_PROGRESS) { log.info("SyncGroup failed: The group began another rebalance. Need to re-join the group. " + "Sent generation was {}", sentGeneration); - // consumer didn't get assignment in this generation, so we need to reset generation - // to avoid joinGroup with out-of-data ownedPartitions in cooperative rebalance - resetStateOnResponseError(ApiKeys.SYNC_GROUP, error, false); future.raise(error); } else if (error == Errors.FENCED_INSTANCE_ID) { // for sync-group request, even if the generation has changed we would not expect the instance id @@ -1114,7 +1125,7 @@ public synchronized RequestFuture maybeLeaveGroup(String leaveReason) { generation.memberId, coordinator, leaveReason); LeaveGroupRequest.Builder request = new LeaveGroupRequest.Builder( rebalanceConfig.groupId, - Collections.singletonList(new MemberIdentity().setMemberId(generation.memberId).setReason(leaveReason)) + Collections.singletonList(new MemberIdentity().setMemberId(generation.memberId).setReason(JoinGroupRequest.maybeTruncateReason(leaveReason))) ); future = client.send(coordinator, request).compose(new LeaveGroupResponseHandler(generation)); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index b853ff99e8231..5eb7d87819795 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals; -import java.time.Duration; import java.util.SortedSet; import java.util.TreeSet; import org.apache.kafka.clients.GroupRebalanceConfig; @@ -141,6 +140,12 @@ private boolean sameRequest(final Set currentRequest, final Gene } private final RebalanceProtocol protocol; + // pending commit offset request in onJoinPrepare + private RequestFuture autoCommitOffsetRequestFuture = null; + // a timer for join prepare to know when to stop. + // it'll set to rebalance timeout so that the member can join the group successfully + // even though offset commit failed. + private Timer joinPrepareTimer = null; /** * Initialize the coordination manager. @@ -483,10 +488,14 @@ void maybeUpdateSubscriptionMetadata() { } } - private boolean coordinatorUnknownAndUnready(Timer timer) { + private boolean coordinatorUnknownAndUnreadySync(Timer timer) { return coordinatorUnknown() && !ensureCoordinatorReady(timer); } + private boolean coordinatorUnknownAndUnreadyAsync() { + return coordinatorUnknown() && !ensureCoordinatorReadyAsync(); + } + /** * Poll for coordinator events. This ensures that the coordinator is known and that the consumer * has joined the group (if it is using group management). This also handles periodic offset commits @@ -512,7 +521,7 @@ public boolean poll(Timer timer, boolean waitForJoinGroup) { // Always update the heartbeat last poll time so that the heartbeat thread does not leave the // group proactively due to application inactivity even if (say) the coordinator cannot be found. pollHeartbeat(timer.currentTimeMs()); - if (coordinatorUnknownAndUnready(timer)) { + if (coordinatorUnknownAndUnreadySync(timer)) { return false; } @@ -740,24 +749,59 @@ private void validateCooperativeAssignment(final Map future = maybeAutoCommitOffsetsAsync(); - // return true when - // 1. future is null, which means no commit request sent, so it is still considered completed - // 2. offset commit completed - // 3. offset commit failed with non-retriable exception - if (future == null) - onJoinPrepareAsyncCommitCompleted = true; - else if (future.succeeded()) - onJoinPrepareAsyncCommitCompleted = true; - else if (future.failed() && !future.isRetriable()) { - log.error("Asynchronous auto-commit of offsets failed: {}", future.exception().getMessage()); - onJoinPrepareAsyncCommitCompleted = true; + // and there is no in-flight offset commit request + if (autoCommitEnabled && autoCommitOffsetRequestFuture == null) { + maybeMarkPartitionsPendingRevocation(); + autoCommitOffsetRequestFuture = maybeAutoCommitOffsetsAsync(); } + // wait for commit offset response before timer expired + if (autoCommitOffsetRequestFuture != null) { + Timer pollTimer = timer.remainingMs() < joinPrepareTimer.remainingMs() ? + timer : joinPrepareTimer; + client.poll(autoCommitOffsetRequestFuture, pollTimer); + joinPrepareTimer.update(); + + // Keep retrying/waiting the offset commit when: + // 1. offset commit haven't done (and joinPrepareTimer not expired) + // 2. failed with retryable exception (and joinPrepareTimer not expired) + // Otherwise, continue to revoke partitions, ex: + // 1. if joinPrepareTime has expired + // 2. if offset commit failed with no-retryable exception + // 3. if offset commit success + boolean onJoinPrepareAsyncCommitCompleted = true; + if (joinPrepareTimer.isExpired()) { + log.error("Asynchronous auto-commit of offsets failed: joinPrepare timeout. Will continue to join group"); + } else if (!autoCommitOffsetRequestFuture.isDone()) { + onJoinPrepareAsyncCommitCompleted = false; + } else if (autoCommitOffsetRequestFuture.failed() && autoCommitOffsetRequestFuture.isRetriable()) { + log.debug("Asynchronous auto-commit of offsets failed with retryable error: {}. Will retry it.", + autoCommitOffsetRequestFuture.exception().getMessage()); + onJoinPrepareAsyncCommitCompleted = false; + } else if (autoCommitOffsetRequestFuture.failed() && !autoCommitOffsetRequestFuture.isRetriable()) { + log.error("Asynchronous auto-commit of offsets failed: {}. Will continue to join group.", + autoCommitOffsetRequestFuture.exception().getMessage()); + } + if (autoCommitOffsetRequestFuture.isDone()) { + autoCommitOffsetRequestFuture = null; + } + if (!onJoinPrepareAsyncCommitCompleted) { + pollTimer.sleep(Math.min(pollTimer.remainingMs(), rebalanceConfig.retryBackoffMs)); + timer.update(); + return false; + } + } // the generation / member-id can possibly be reset by the heartbeat thread // upon getting errors or heartbeat timeouts; in this case whatever is previously @@ -809,11 +853,30 @@ else if (future.failed() && !future.isRetriable()) { isLeader = false; subscriptions.resetGroupSubscription(); + joinPrepareTimer = null; + autoCommitOffsetRequestFuture = null; + timer.update(); if (exception != null) { throw new KafkaException("User rebalance callback throws an error", exception); } - return onJoinPrepareAsyncCommitCompleted; + return true; + } + + private void maybeMarkPartitionsPendingRevocation() { + if (protocol != RebalanceProtocol.EAGER) { + return; + } + + // When asynchronously committing offsets prior to the revocation of a set of partitions, there will be a + // window of time between when the offset commit is sent and when it returns and revocation completes. It is + // possible for pending fetches for these partitions to return during this time, which means the application's + // position may get ahead of the committed position prior to revocation. This can cause duplicate consumption. + // To prevent this, we mark the partitions as "pending revocation," which stops the Fetcher from sending new + // fetches or returning data from previous fetches to the user. + Set partitions = subscriptions.assignedPartitions(); + log.debug("Marking assigned partitions pending for revocation: {}", partitions); + subscriptions.markPendingRevocation(partitions); } @Override @@ -1009,7 +1072,7 @@ public RequestFuture commitOffsetsAsync(final Map offsets, return true; do { - if (coordinatorUnknownAndUnready(timer)) { + if (coordinatorUnknownAndUnreadySync(timer)) { return false; } @@ -1229,8 +1292,10 @@ RequestFuture sendOffsetCommitRequest(final Map sendOffsetCommitRequest(final Map sendOffsetCommitRequest(final Map(requestTopicDataMap.values())) ); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java index 4b9112016e986..6646dc6c893e0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -211,8 +211,23 @@ public void poll(RequestFuture future) { * @throws InterruptException if the calling thread is interrupted */ public boolean poll(RequestFuture future, Timer timer) { + return poll(future, timer, false); + } + + /** + * Block until the provided request future request has finished or the timeout has expired. + * + * @param future The request future to wait for + * @param timer Timer bounding how long this method can block + * @param disableWakeup true if we should not check for wakeups, false otherwise + * + * @return true if the future is done, false otherwise + * @throws WakeupException if {@link #wakeup()} is called from another thread and `disableWakeup` is false + * @throws InterruptException if the calling thread is interrupted + */ + public boolean poll(RequestFuture future, Timer timer, boolean disableWakeup) { do { - poll(timer, future); + poll(timer, future, disableWakeup); } while (!future.isDone() && timer.notExpired()); return future.isDone(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 73ffd217efe21..c93b675f755f2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -346,6 +346,7 @@ public void onFailure(RuntimeException e) { FetchSessionHandler handler = sessionHandler(fetchTarget.id()); if (handler != null) { handler.handleError(e); + handler.sessionTopicPartitions().forEach(subscriptions::clearPreferredReadReplica); } } finally { nodesWithPendingFetchRequests.remove(fetchTarget.id()); @@ -1154,7 +1155,9 @@ Node selectReadReplica(TopicPartition partition, Node leaderReplica, long curren } else { log.trace("Not fetching from {} for partition {} since it is marked offline or is missing from our metadata," + " using the leader instead.", nodeId, partition); - subscriptions.clearPreferredReadReplica(partition); + // Note that this condition may happen due to stale metadata, so we clear preferred replica and + // refresh metadata. + requestMetadataUpdate(partition); return leaderReplica; } } else { @@ -1335,16 +1338,16 @@ private CompletedFetch initializeCompletedFetch(CompletedFetch nextCompletedFetc error == Errors.FENCED_LEADER_EPOCH || error == Errors.OFFSET_NOT_AVAILABLE) { log.debug("Error in fetch for partition {}: {}", tp, error.exceptionName()); - this.metadata.requestUpdate(); + requestMetadataUpdate(tp); } else if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) { log.warn("Received unknown topic or partition error in fetch for partition {}", tp); - this.metadata.requestUpdate(); + requestMetadataUpdate(tp); } else if (error == Errors.UNKNOWN_TOPIC_ID) { log.warn("Received unknown topic ID error in fetch for partition {}", tp); - this.metadata.requestUpdate(); + requestMetadataUpdate(tp); } else if (error == Errors.INCONSISTENT_TOPIC_ID) { log.warn("Received inconsistent topic ID error in fetch for partition {}", tp); - this.metadata.requestUpdate(); + requestMetadataUpdate(tp); } else if (error == Errors.OFFSET_OUT_OF_RANGE) { Optional clearedReplicaId = subscriptions.clearPreferredReadReplica(tp); if (!clearedReplicaId.isPresent()) { @@ -1944,4 +1947,9 @@ private Set topicsForPartitions(Collection partitions) { return partitions.stream().map(TopicPartition::topic).collect(Collectors.toSet()); } + private void requestMetadataUpdate(TopicPartition topicPartition) { + this.metadata.requestUpdate(); + this.subscriptions.clearPreferredReadReplica(topicPartition); + } + } 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 0225822a19466..b41122f0e5060 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 @@ -620,7 +620,12 @@ public synchronized Optional preferredReadReplica(TopicPartition tp, lo * @return the removed preferred read replica if set, None otherwise. */ public synchronized Optional clearPreferredReadReplica(TopicPartition tp) { - return assignedState(tp).clearPreferredReadReplica(); + final TopicPartitionState topicPartitionState = assignedStateOrNull(tp); + if (topicPartitionState == null) { + return Optional.empty(); + } else { + return topicPartitionState.clearPreferredReadReplica(); + } } public synchronized Map allConsumed() { @@ -738,6 +743,10 @@ public synchronized void pause(TopicPartition tp) { assignedState(tp).pause(); } + public synchronized void markPendingRevocation(Set tps) { + tps.forEach(tp -> assignedState(tp).markPendingRevocation()); + } + public synchronized void resume(TopicPartition tp) { assignedState(tp).resume(); } @@ -769,6 +778,7 @@ private static class TopicPartitionState { private Long logStartOffset; // the log start offset private Long lastStableOffset; private boolean paused; // whether this partition has been paused by the user + private boolean pendingRevocation; private OffsetResetStrategy resetStrategy; // the strategy to use if the offset needs resetting private Long nextRetryTimeMs; private Integer preferredReadReplica; @@ -777,6 +787,7 @@ private static class TopicPartitionState { TopicPartitionState() { this.paused = false; + this.pendingRevocation = false; this.endOffsetRequested = false; this.fetchState = FetchStates.INITIALIZING; this.position = null; @@ -966,12 +977,16 @@ private void pause() { this.paused = true; } + private void markPendingRevocation() { + this.pendingRevocation = true; + } + private void resume() { this.paused = false; } private boolean isFetchable() { - return !paused && hasValidPosition(); + return !paused && !pendingRevocation && hasValidPosition(); } private void highWatermark(Long highWatermark) { diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 74d408d9a5a49..148023e3a56ef 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -425,8 +425,11 @@ private void warnIfPartitionerDeprecated() { enableAdaptivePartitioning, config.getLong(ProducerConfig.PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG) ); + // As per Kafka producer configuration documentation batch.size may be set to 0 to explicitly disable + // batching which in practice actually means using a batch size of 1. + int batchSize = Math.max(1, config.getInt(ProducerConfig.BATCH_SIZE_CONFIG)); this.accumulator = new RecordAccumulator(logContext, - config.getInt(ProducerConfig.BATCH_SIZE_CONFIG), + batchSize, this.compressionType, lingerMs(config), retryBackoffMs, @@ -437,7 +440,7 @@ private void warnIfPartitionerDeprecated() { time, apiVersions, transactionManager, - new BufferPool(this.totalMemorySize, config.getInt(ProducerConfig.BATCH_SIZE_CONFIG), metrics, time, PRODUCER_METRIC_GROUP_NAME)); + new BufferPool(this.totalMemorySize, batchSize, metrics, time, PRODUCER_METRIC_GROUP_NAME)); List addresses = ClientUtils.parseAndValidateAddresses( config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG), @@ -1465,13 +1468,21 @@ public boolean isDone() { private class AppendCallbacks implements RecordAccumulator.AppendCallbacks { private final Callback userCallback; private final ProducerInterceptors interceptors; - private final ProducerRecord record; - protected int partition = RecordMetadata.UNKNOWN_PARTITION; + private final String topic; + private final Integer recordPartition; + private final String recordLogString; + private volatile int partition = RecordMetadata.UNKNOWN_PARTITION; + private volatile TopicPartition topicPartition; private AppendCallbacks(Callback userCallback, ProducerInterceptors interceptors, ProducerRecord record) { this.userCallback = userCallback; this.interceptors = interceptors; - this.record = record; + // Extract record info as we don't want to keep a reference to the record during + // whole lifetime of the batch. + // We don't want to have an NPE here, because the interceptors would not be notified (see .doSend). + topic = record != null ? record.topic() : null; + recordPartition = record != null ? record.partition() : null; + recordLogString = log.isTraceEnabled() && record != null ? record.toString() : ""; } @Override @@ -1491,7 +1502,7 @@ public void setPartition(int partition) { if (log.isTraceEnabled()) { // Log the message here, because we don't know the partition before that. - log.trace("Attempting to append record {} with callback {} to topic {} partition {}", record, userCallback, record.topic(), partition); + log.trace("Attempting to append record {} with callback {} to topic {} partition {}", recordLogString, userCallback, topic, partition); } } @@ -1500,11 +1511,15 @@ public int getPartition() { } public TopicPartition topicPartition() { - if (record == null) - return null; - return partition == RecordMetadata.UNKNOWN_PARTITION - ? ProducerInterceptors.extractTopicPartition(record) - : new TopicPartition(record.topic(), partition); + if (topicPartition == null && topic != null) { + if (partition != RecordMetadata.UNKNOWN_PARTITION) + topicPartition = new TopicPartition(topic, partition); + else if (recordPartition != null) + topicPartition = new TopicPartition(topic, recordPartition); + else + topicPartition = new TopicPartition(topic, RecordMetadata.UNKNOWN_PARTITION); + } + return topicPartition; } } } 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 aff5e49cfcb1b..a3ae15a3baf0d 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 @@ -105,7 +105,7 @@ public class ProducerConfig extends AbstractConfig { "If a broker cannot process produce requests from a partition for " + PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG + " time, " + "the partitioner treats that partition as not available. If the value is 0, this logic is disabled. " + "Note: this setting has no effect if a custom partitioner is used or " + PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG - + " is set to 'false'"; + + " is set to 'false'"; /** partitioner.ignore.keys */ public static final String PARTITIONER_IGNORE_KEYS_CONFIG = "partitioner.ignore.keys"; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BuiltInPartitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BuiltInPartitioner.java index a5805df56b78d..a1493d2edd66a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BuiltInPartitioner.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BuiltInPartitioner.java @@ -56,6 +56,9 @@ public class BuiltInPartitioner { public BuiltInPartitioner(LogContext logContext, String topic, int stickyBatchSize) { this.log = logContext.logger(BuiltInPartitioner.class); this.topic = topic; + if (stickyBatchSize < 1) { + throw new IllegalArgumentException("stickyBatchSize must be >= 1 but got " + stickyBatchSize); + } this.stickyBatchSize = stickyBatchSize; } @@ -170,13 +173,52 @@ boolean isPartitionChanged(StickyPartitionInfo partitionInfo) { * @param cluster The cluster information */ void updatePartitionInfo(StickyPartitionInfo partitionInfo, int appendedBytes, Cluster cluster) { + updatePartitionInfo(partitionInfo, appendedBytes, cluster, true); + } + + /** + * Update partition info with the number of bytes appended and maybe switch partition. + * NOTE this function needs to be called under the partition's batch queue lock. + * + * @param partitionInfo The sticky partition info object returned by peekCurrentPartitionInfo + * @param appendedBytes The number of bytes appended to this partition + * @param cluster The cluster information + * @param enableSwitch If true, switch partition once produced enough bytes + */ + void updatePartitionInfo(StickyPartitionInfo partitionInfo, int appendedBytes, Cluster cluster, boolean enableSwitch) { // partitionInfo may be null if the caller didn't use built-in partitioner. if (partitionInfo == null) return; assert partitionInfo == stickyPartitionInfo.get(); int producedBytes = partitionInfo.producedBytes.addAndGet(appendedBytes); - if (producedBytes >= stickyBatchSize) { + + // We're trying to switch partition once we produce stickyBatchSize bytes to a partition + // but doing so may hinder batching because partition switch may happen while batch isn't + // ready to send. This situation is especially likely with high linger.ms setting. + // Consider the following example: + // linger.ms=500, producer produces 12KB in 500ms, batch.size=16KB + // - first batch collects 12KB in 500ms, gets sent + // - second batch collects 4KB, then we switch partition, so 4KB gets eventually sent + // - ... and so on - we'd get 12KB and 4KB batches + // To get more optimal batching and avoid 4KB fractional batches, the caller may disallow + // partition switch if batch is not ready to send, so with the example above we'd avoid + // fractional 4KB batches: in that case the scenario would look like this: + // - first batch collects 12KB in 500ms, gets sent + // - second batch collects 4KB, but partition switch doesn't happen because batch in not ready + // - second batch collects 12KB in 500ms, gets sent and now we switch partition. + // - ... and so on - we'd just send 12KB batches + // We cap the produced bytes to not exceed 2x of the batch size to avoid pathological cases + // (e.g. if we have a mix of keyed and unkeyed messages, key messages may create an + // unready batch after the batch that disabled partition switch becomes ready). + // As a result, with high latency.ms setting we end up switching partitions after producing + // between stickyBatchSize and stickyBatchSize * 2 bytes, to better align with batch boundary. + if (producedBytes >= stickyBatchSize * 2) { + log.trace("Produced {} bytes, exceeding twice the batch size of {} bytes, with switching set to {}", + producedBytes, stickyBatchSize, enableSwitch); + } + + if (producedBytes >= stickyBatchSize && enableSwitch || producedBytes >= stickyBatchSize * 2) { // We've produced enough to this partition, switch to next. StickyPartitionInfo newPartitionInfo = new StickyPartitionInfo(nextPartition(cluster)); stickyPartitionInfo.set(newPartitionInfo); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 4168ea68aa430..983426316df9e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -213,6 +213,43 @@ private void setPartition(AppendCallbacks callbacks, int partition) { callbacks.setPartition(partition); } + /** + * Check if partition concurrently changed, or we need to complete previously disabled partition change. + * + * @param topic The topic + * @param topicInfo The topic info + * @param partitionInfo The built-in partitioner's partition info + * @param deque The partition queue + * @param nowMs The current time, in milliseconds + * @param cluster THe cluster metadata + * @return 'true' if partition changed and we need to get new partition info and retry, + * 'false' otherwise + */ + private boolean partitionChanged(String topic, + TopicInfo topicInfo, + BuiltInPartitioner.StickyPartitionInfo partitionInfo, + Deque deque, long nowMs, + Cluster cluster) { + if (topicInfo.builtInPartitioner.isPartitionChanged(partitionInfo)) { + log.trace("Partition {} for topic {} switched by a concurrent append, retrying", + partitionInfo.partition(), topic); + return true; + } + + // We might have disabled partition switch if the queue had incomplete batches. + // Check if all batches are full now and switch . + if (allBatchesFull(deque)) { + topicInfo.builtInPartitioner.updatePartitionInfo(partitionInfo, 0, cluster, true); + if (topicInfo.builtInPartitioner.isPartitionChanged(partitionInfo)) { + log.trace("Completed previously disabled switch for topic {} partition {}, retrying", + topic, partitionInfo.partition()); + return true; + } + } + + return false; + } + /** * Add a record to the accumulator, return the append result *

@@ -275,14 +312,14 @@ public RecordAppendResult append(String topic, Deque dq = topicInfo.batches.computeIfAbsent(effectivePartition, k -> new ArrayDeque<>()); synchronized (dq) { // After taking the lock, validate that the partition hasn't changed and retry. - if (topicInfo.builtInPartitioner.isPartitionChanged(partitionInfo)) { - log.trace("Partition {} for topic {} switched by a concurrent append, retrying", - partitionInfo.partition(), topic); + if (partitionChanged(topic, topicInfo, partitionInfo, dq, nowMs, cluster)) continue; - } + RecordAppendResult appendResult = tryAppend(timestamp, key, value, headers, callbacks, dq, nowMs); if (appendResult != null) { - topicInfo.builtInPartitioner.updatePartitionInfo(partitionInfo, appendResult.appendedBytes, cluster); + // If queue has incomplete batches we disable switch (see comments in updatePartitionInfo). + boolean enableSwitch = allBatchesFull(dq); + topicInfo.builtInPartitioner.updatePartitionInfo(partitionInfo, appendResult.appendedBytes, cluster, enableSwitch); return appendResult; } } @@ -297,21 +334,26 @@ public RecordAppendResult append(String topic, byte maxUsableMagic = apiVersions.maxUsableProduceMagic(); int size = Math.max(this.batchSize, AbstractRecords.estimateSizeInBytesUpperBound(maxUsableMagic, compression, key, value, headers)); log.trace("Allocating a new {} byte message buffer for topic {} partition {} with remaining timeout {}ms", size, topic, partition, maxTimeToBlock); + // This call may block if we exhausted buffer space. buffer = free.allocate(size, maxTimeToBlock); + // Update the current time in case the buffer allocation blocked above. + // NOTE: getting time may be expensive, so calling it under a lock + // should be avoided. + nowMs = time.milliseconds(); } synchronized (dq) { // After taking the lock, validate that the partition hasn't changed and retry. - if (topicInfo.builtInPartitioner.isPartitionChanged(partitionInfo)) { - log.trace("Partition {} for topic {} switched by a concurrent append, retrying", - partitionInfo.partition(), topic); + if (partitionChanged(topic, topicInfo, partitionInfo, dq, nowMs, cluster)) continue; - } - RecordAppendResult appendResult = appendNewBatch(topic, effectivePartition, dq, timestamp, key, value, headers, callbacks, buffer); + + RecordAppendResult appendResult = appendNewBatch(topic, effectivePartition, dq, timestamp, key, value, headers, callbacks, buffer, nowMs); // Set buffer to null, so that deallocate doesn't return it back to free pool, since it's used in the batch. if (appendResult.newBatchCreated) buffer = null; - topicInfo.builtInPartitioner.updatePartitionInfo(partitionInfo, appendResult.appendedBytes, cluster); + // If queue has incomplete batches we disable switch (see comments in updatePartitionInfo). + boolean enableSwitch = allBatchesFull(dq); + topicInfo.builtInPartitioner.updatePartitionInfo(partitionInfo, appendResult.appendedBytes, cluster, enableSwitch); return appendResult; } } @@ -333,6 +375,7 @@ public RecordAppendResult append(String topic, * @param headers the Headers for the record * @param callbacks The callbacks to execute * @param buffer The buffer for the new batch + * @param nowMs The current time, in milliseconds */ private RecordAppendResult appendNewBatch(String topic, int partition, @@ -342,11 +385,10 @@ private RecordAppendResult appendNewBatch(String topic, byte[] value, Header[] headers, AppendCallbacks callbacks, - ByteBuffer buffer) { + ByteBuffer buffer, + long nowMs) { assert partition != RecordMetadata.UNKNOWN_PARTITION; - // Update the current time in case the buffer allocation blocked above. - long nowMs = time.milliseconds(); RecordAppendResult appendResult = tryAppend(timestamp, key, value, headers, callbacks, dq, nowMs); if (appendResult != null) { // Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen often... @@ -373,6 +415,15 @@ private MemoryRecordsBuilder recordsBuilder(ByteBuffer buffer, byte maxUsableMag } /** + * Check if all batches in the queue are full. + */ + private boolean allBatchesFull(Deque deque) { + // Only the last batch may be incomplete, so we just check that. + ProducerBatch last = deque.peekLast(); + return last == null || last.isFull(); + } + + /** * Try to append to a ProducerBatch. * * If it is full, we return null and a new batch is created. We also close the batch for record appends to free up 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 5aab62eaf225a..de5a6ced41c85 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 @@ -1290,7 +1290,7 @@ public void handleResponse(AbstractResponse response) { } else if (error == Errors.NOT_COORDINATOR || error == Errors.COORDINATOR_NOT_AVAILABLE) { lookupCoordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); reenqueue(); - } else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.CONCURRENT_TRANSACTIONS) { + } else if (error.exception() instanceof RetriableException) { reenqueue(); } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED || error == Errors.CLUSTER_AUTHORIZATION_FAILED) { @@ -1347,7 +1347,7 @@ public void handleResponse(AbstractResponse response) { maybeOverrideRetryBackoffMs(); reenqueue(); return; - } else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.UNKNOWN_TOPIC_OR_PARTITION) { + } else if (error.exception() instanceof RetriableException) { reenqueue(); return; } else if (error == Errors.INVALID_PRODUCER_EPOCH || error == Errors.PRODUCER_FENCED) { @@ -1467,7 +1467,7 @@ public void handleResponse(AbstractResponse response) { } result.done(); log.info("Discovered {} coordinator {}", coordinatorType.toString().toLowerCase(Locale.ROOT), node); - } else if (error == Errors.COORDINATOR_NOT_AVAILABLE) { + } else if (error.exception() instanceof RetriableException) { reenqueue(); } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) { fatalError(error.exception()); @@ -1515,7 +1515,7 @@ public void handleResponse(AbstractResponse response) { } else if (error == Errors.COORDINATOR_NOT_AVAILABLE || error == Errors.NOT_COORDINATOR) { lookupCoordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); reenqueue(); - } else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.CONCURRENT_TRANSACTIONS) { + } else if (error.exception() instanceof RetriableException) { reenqueue(); } else if (error == Errors.INVALID_PRODUCER_EPOCH || error == Errors.PRODUCER_FENCED) { // We could still receive INVALID_PRODUCER_EPOCH from old versioned transaction coordinator, @@ -1572,7 +1572,7 @@ public void handleResponse(AbstractResponse response) { } else if (error == Errors.COORDINATOR_NOT_AVAILABLE || error == Errors.NOT_COORDINATOR) { lookupCoordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); reenqueue(); - } else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.CONCURRENT_TRANSACTIONS) { + } else if (error.exception() instanceof RetriableException) { reenqueue(); } else if (error == Errors.UNKNOWN_PRODUCER_ID || error == Errors.INVALID_PRODUCER_ID_MAPPING) { abortableErrorIfPossible(error.exception()); diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index 7d3f6f08a0139..84b77ef5f40db 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -252,7 +252,12 @@ public Node nodeById(int id) { */ public Optional nodeIfOnline(TopicPartition partition, int id) { Node node = nodeById(id); - if (node != null && !Arrays.asList(partition(partition).offlineReplicas()).contains(node)) { + PartitionInfo partitionInfo = partition(partition); + + if (node != null && partitionInfo != null && + !Arrays.asList(partitionInfo.offlineReplicas()).contains(node) && + Arrays.asList(partitionInfo.replicas()).contains(node)) { + return Optional.of(node); } else { return Optional.empty(); diff --git a/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java index 0b90da8f800a6..8b7a9649c2c43 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java @@ -36,6 +36,8 @@ public class BrokerSecurityConfigs { public static final String SASL_SERVER_CALLBACK_HANDLER_CLASS = "sasl.server.callback.handler.class"; public static final String SSL_PRINCIPAL_MAPPING_RULES_CONFIG = "ssl.principal.mapping.rules"; public static final String CONNECTIONS_MAX_REAUTH_MS = "connections.max.reauth.ms"; + public static final int DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE = 524288; + public static final String SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG = "sasl.server.max.receive.size"; public static final String PRINCIPAL_BUILDER_CLASS_DOC = "The fully qualified name of a class that implements the " + "KafkaPrincipalBuilder interface, which is used to build the KafkaPrincipal object used during " + @@ -89,4 +91,8 @@ public class BrokerSecurityConfigs { + "The broker will disconnect any such connection that is not re-authenticated within the session lifetime and that is then subsequently " + "used for any purpose other than re-authentication. Configuration names can optionally be prefixed with listener prefix and SASL " + "mechanism name in lower-case. For example, listener.name.sasl_ssl.oauthbearer.connections.max.reauth.ms=3600000"; + + public static final String SASL_SERVER_MAX_RECEIVE_SIZE_DOC = "The maximum receive size allowed before and during initial SASL authentication." + + " Default receive size is 512KB. GSSAPI limits requests to 64K, but we allow upto 512KB by default for custom SASL mechanisms. In practice," + + " PLAIN, SCRAM and OAUTH mechanisms can use much smaller limits."; } diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ConcurrentTransactionsException.java b/clients/src/main/java/org/apache/kafka/common/errors/ConcurrentTransactionsException.java index 6ad6b8a3ddb4c..118b4de50aadf 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/ConcurrentTransactionsException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/ConcurrentTransactionsException.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.common.errors; -public class ConcurrentTransactionsException extends ApiException { +public class ConcurrentTransactionsException extends RetriableException { private static final long serialVersionUID = 1L; public ConcurrentTransactionsException(final String message) { diff --git a/clients/src/main/java/org/apache/kafka/common/internals/Topic.java b/clients/src/main/java/org/apache/kafka/common/internals/Topic.java index fbf491bab9800..fe4a01c52a635 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/Topic.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/Topic.java @@ -28,8 +28,11 @@ public class Topic { public static final String GROUP_METADATA_TOPIC_NAME = "__consumer_offsets"; public static final String TRANSACTION_STATE_TOPIC_NAME = "__transaction_state"; - public static final String METADATA_TOPIC_NAME = "__cluster_metadata"; - public static final TopicPartition METADATA_TOPIC_PARTITION = new TopicPartition(METADATA_TOPIC_NAME, 0); + public static final String CLUSTER_METADATA_TOPIC_NAME = "__cluster_metadata"; + public static final TopicPartition CLUSTER_METADATA_TOPIC_PARTITION = new TopicPartition( + CLUSTER_METADATA_TOPIC_NAME, + 0 + ); public static final String LEGAL_CHARS = "[a-zA-Z0-9._-]"; private static final Set INTERNAL_TOPICS = Collections.unmodifiableSet( @@ -43,17 +46,31 @@ public static void validate(String topic) { }); } - public static void validate(String name, String logPrefix, Consumer throwableConsumer) { + private static String detectInvalidTopic(String name) { if (name.isEmpty()) - throwableConsumer.accept(logPrefix + " is illegal, it can't be empty"); - if (".".equals(name) || "..".equals(name)) - throwableConsumer.accept(logPrefix + " cannot be \".\" or \"..\""); + return "the empty string is not allowed"; + if (".".equals(name)) + return "'.' is not allowed"; + if ("..".equals(name)) + return "'..' is not allowed"; if (name.length() > MAX_NAME_LENGTH) - throwableConsumer.accept(logPrefix + " is illegal, it can't be longer than " + MAX_NAME_LENGTH + - " characters, " + logPrefix + ": " + name); + return "the length of '" + name + "' is longer than the max allowed length " + MAX_NAME_LENGTH; if (!containsValidPattern(name)) - throwableConsumer.accept(logPrefix + " \"" + name + "\" is illegal, it contains a character other than " + - "ASCII alphanumerics, '.', '_' and '-'"); + return "'" + name + "' contains one or more characters other than " + + "ASCII alphanumerics, '.', '_' and '-'"; + return null; + } + + public static boolean isValid(String name) { + String reasonInvalid = detectInvalidTopic(name); + return reasonInvalid == null; + } + + public static void validate(String name, String logPrefix, Consumer throwableConsumer) { + String reasonInvalid = detectInvalidTopic(name); + if (reasonInvalid != null) { + throwableConsumer.accept(logPrefix + " is invalid: " + reasonInvalid); + } } public static boolean isInternal(String topic) { diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java index c6b8574186a88..09b7c05c8f283 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java @@ -91,7 +91,10 @@ public long windowSize(MetricConfig config, long now) { if (numFullWindows < minFullWindows) totalElapsedTimeMs += (minFullWindows - numFullWindows) * config.timeWindowMs(); - return totalElapsedTimeMs; + // If window size is being calculated at the exact beginning of the window with no prior samples, the window size + // will result in a value of 0. Calculation of rate over a window is size 0 is undefined, hence, we assume the + // minimum window size to be at least 1ms. + return Math.max(totalElapsedTimeMs, 1); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 628c9407cc2dd..f727bd18e5fb9 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -231,7 +231,7 @@ private static String toHtml() { b.append("Name\n"); b.append("Key\n"); b.append(""); - for (ApiKeys key : zkBrokerApis()) { + for (ApiKeys key : clientApis()) { b.append("\n"); b.append(""); b.append("" + key.name + ""); @@ -271,15 +271,22 @@ public static EnumSet controllerApis() { return apisForListener(ApiMessageType.ListenerType.CONTROLLER); } + public static EnumSet clientApis() { + List apis = Arrays.stream(ApiKeys.values()) + .filter(apiKey -> apiKey.inScope(ApiMessageType.ListenerType.ZK_BROKER) || apiKey.inScope(ApiMessageType.ListenerType.BROKER)) + .collect(Collectors.toList()); + return EnumSet.copyOf(apis); + } + public static EnumSet apisForListener(ApiMessageType.ListenerType listener) { return APIS_BY_LISTENER.get(listener); } private static EnumSet filterApisForListener(ApiMessageType.ListenerType listener) { - List controllerApis = Arrays.stream(ApiKeys.values()) - .filter(apiKey -> apiKey.messageType.listeners().contains(listener)) + List apis = Arrays.stream(ApiKeys.values()) + .filter(apiKey -> apiKey.inScope(listener)) .collect(Collectors.toList()); - return EnumSet.copyOf(controllerApis); + return EnumSet.copyOf(apis); } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ByteBufferAccessor.java b/clients/src/main/java/org/apache/kafka/common/protocol/ByteBufferAccessor.java index bd0925d6db358..f643f5b5779b1 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ByteBufferAccessor.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ByteBufferAccessor.java @@ -54,8 +54,15 @@ public double readDouble() { } @Override - public void readArray(byte[] arr) { + public byte[] readArray(int size) { + int remaining = buf.remaining(); + if (size > remaining) { + throw new RuntimeException("Error reading byte array of " + size + " byte(s): only " + remaining + + " byte(s) available"); + } + byte[] arr = new byte[size]; buf.get(arr); + return arr; } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/DataInputStreamReadable.java b/clients/src/main/java/org/apache/kafka/common/protocol/DataInputStreamReadable.java deleted file mode 100644 index 70ed52d6a02f6..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/protocol/DataInputStreamReadable.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.common.protocol; - -import org.apache.kafka.common.utils.ByteUtils; - -import java.io.Closeable; -import java.io.DataInputStream; -import java.io.IOException; -import java.nio.ByteBuffer; - -public class DataInputStreamReadable implements Readable, Closeable { - protected final DataInputStream input; - - public DataInputStreamReadable(DataInputStream input) { - this.input = input; - } - - @Override - public byte readByte() { - try { - return input.readByte(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public short readShort() { - try { - return input.readShort(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public int readInt() { - try { - return input.readInt(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public long readLong() { - try { - return input.readLong(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public double readDouble() { - try { - return input.readDouble(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public void readArray(byte[] arr) { - try { - input.readFully(arr); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public int readUnsignedVarint() { - try { - return ByteUtils.readUnsignedVarint(input); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public ByteBuffer readByteBuffer(int length) { - byte[] arr = new byte[length]; - readArray(arr); - return ByteBuffer.wrap(arr); - } - - @Override - public int readVarint() { - try { - return ByteUtils.readVarint(input); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public long readVarlong() { - try { - return ByteUtils.readVarlong(input); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public int remaining() { - try { - return input.available(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public void close() { - try { - input.close(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - -} 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 2ca42bafcfb86..c220bbcde43cb 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 @@ -132,6 +132,8 @@ import java.util.HashMap; import java.util.Map; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; import java.util.function.Function; /** @@ -469,7 +471,8 @@ public static Errors forCode(short code) { * If there are multiple matches in the class hierarchy, the first match starting from the bottom is used. */ public static Errors forException(Throwable t) { - Class clazz = t.getClass(); + Throwable cause = maybeUnwrapException(t); + Class clazz = cause.getClass(); while (clazz != null) { Errors error = classToError.get(clazz); if (error != null) @@ -479,6 +482,22 @@ public static Errors forException(Throwable t) { return UNKNOWN_SERVER_ERROR; } + /** + * Check if a Throwable is a commonly wrapped exception type (e.g. `CompletionException`) and return + * the cause if so. This is useful to handle cases where exceptions may be raised from a future or a + * completion stage (as might be the case for requests sent to the controller in `ControllerApis`). + * + * @param t The Throwable to check + * @return The throwable itself or its cause if it is an instance of a commonly wrapped exception type + */ + public static Throwable maybeUnwrapException(Throwable t) { + if (t instanceof CompletionException || t instanceof ExecutionException) { + return t.getCause(); + } else { + return t; + } + } + private static String toHtml() { final StringBuilder b = new StringBuilder(); b.append("\n"); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index a75eb0661d102..2bc0fc6efef7d 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -133,7 +133,7 @@ public static String toHtml() { b.append("\n"); schemaToFieldTableHtml(ResponseHeaderData.SCHEMAS[i], b); } - for (ApiKeys key : ApiKeys.zkBrokerApis()) { + for (ApiKeys key : ApiKeys.clientApis()) { // Key b.append("
"); b.append(""); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java b/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java index 561696827df63..80bee86748269 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java @@ -32,7 +32,7 @@ public interface Readable { int readInt(); long readLong(); double readDouble(); - void readArray(byte[] arr); + byte[] readArray(int length); int readUnsignedVarint(); ByteBuffer readByteBuffer(int length); int readVarint(); @@ -40,8 +40,7 @@ public interface Readable { int remaining(); default String readString(int length) { - byte[] arr = new byte[length]; - readArray(arr); + byte[] arr = readArray(length); return new String(arr, StandardCharsets.UTF_8); } @@ -49,8 +48,7 @@ default List readUnknownTaggedField(List unknown if (unknowns == null) { unknowns = new ArrayList<>(); } - byte[] data = new byte[size]; - readArray(data); + byte[] data = readArray(size); unknowns.add(new RawTaggedField(tag, data)); return unknowns; } diff --git a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java index 66a4a14d22bed..a9407217605e9 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordUtils.java @@ -27,10 +27,9 @@ * Utility class for easy interaction with control records. */ public class ControlRecordUtils { - - public static final short LEADER_CHANGE_SCHEMA_HIGHEST_VERSION = LeaderChangeMessage.HIGHEST_SUPPORTED_VERSION; - public static final short SNAPSHOT_HEADER_HIGHEST_VERSION = SnapshotHeaderRecord.HIGHEST_SUPPORTED_VERSION; - public static final short SNAPSHOT_FOOTER_HIGHEST_VERSION = SnapshotFooterRecord.HIGHEST_SUPPORTED_VERSION; + public static final short LEADER_CHANGE_CURRENT_VERSION = 0; + public static final short SNAPSHOT_HEADER_CURRENT_VERSION = 0; + public static final short SNAPSHOT_FOOTER_CURRENT_VERSION = 0; public static LeaderChangeMessage deserializeLeaderChangeMessage(Record record) { ControlRecordType recordType = ControlRecordType.parse(record.key()); @@ -43,7 +42,7 @@ public static LeaderChangeMessage deserializeLeaderChangeMessage(Record record) public static LeaderChangeMessage deserializeLeaderChangeMessage(ByteBuffer data) { ByteBufferAccessor byteBufferAccessor = new ByteBufferAccessor(data.duplicate()); - return new LeaderChangeMessage(byteBufferAccessor, LEADER_CHANGE_SCHEMA_HIGHEST_VERSION); + return new LeaderChangeMessage(byteBufferAccessor, LEADER_CHANGE_CURRENT_VERSION); } public static SnapshotHeaderRecord deserializedSnapshotHeaderRecord(Record record) { @@ -57,7 +56,7 @@ public static SnapshotHeaderRecord deserializedSnapshotHeaderRecord(Record recor public static SnapshotHeaderRecord deserializedSnapshotHeaderRecord(ByteBuffer data) { ByteBufferAccessor byteBufferAccessor = new ByteBufferAccessor(data.duplicate()); - return new SnapshotHeaderRecord(byteBufferAccessor, SNAPSHOT_HEADER_HIGHEST_VERSION); + return new SnapshotHeaderRecord(byteBufferAccessor, SNAPSHOT_HEADER_CURRENT_VERSION); } public static SnapshotFooterRecord deserializedSnapshotFooterRecord(Record record) { @@ -71,6 +70,6 @@ public static SnapshotFooterRecord deserializedSnapshotFooterRecord(Record recor public static SnapshotFooterRecord deserializedSnapshotFooterRecord(ByteBuffer data) { ByteBufferAccessor byteBufferAccessor = new ByteBufferAccessor(data.duplicate()); - return new SnapshotFooterRecord(byteBufferAccessor, SNAPSHOT_FOOTER_HIGHEST_VERSION); + return new SnapshotFooterRecord(byteBufferAccessor, SNAPSHOT_FOOTER_CURRENT_VERSION); } } diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java index 8772556b1dec1..b2235fef49039 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java @@ -342,6 +342,8 @@ private static DefaultRecord readFrom(ByteBuffer buffer, int numHeaders = ByteUtils.readVarint(buffer); if (numHeaders < 0) throw new InvalidRecordException("Found invalid number of record headers " + numHeaders); + if (numHeaders > buffer.remaining()) + throw new InvalidRecordException("Found invalid number of record headers. " + numHeaders + " is larger than the remaining size of the buffer"); final Header[] headers; if (numHeaders == 0) diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java index bd80981d84bda..bc8f32491c0a8 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java @@ -107,7 +107,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe static final int PARTITION_LEADER_EPOCH_LENGTH = 4; static final int MAGIC_OFFSET = PARTITION_LEADER_EPOCH_OFFSET + PARTITION_LEADER_EPOCH_LENGTH; static final int MAGIC_LENGTH = 1; - static final int CRC_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH; + public static final int CRC_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH; static final int CRC_LENGTH = 4; static final int ATTRIBUTES_OFFSET = CRC_OFFSET + CRC_LENGTH; static final int ATTRIBUTE_LENGTH = 2; 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 02fb7d7f12d55..3e9360f04ca69 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 @@ -609,18 +609,27 @@ public void appendLeaderChangeMessage(long timestamp, LeaderChangeMessage leader if (partitionLeaderEpoch == RecordBatch.NO_PARTITION_LEADER_EPOCH) { throw new IllegalArgumentException("Partition leader epoch must be valid, but get " + partitionLeaderEpoch); } - appendControlRecord(timestamp, ControlRecordType.LEADER_CHANGE, - MessageUtil.toByteBuffer(leaderChangeMessage, ControlRecordUtils.LEADER_CHANGE_SCHEMA_HIGHEST_VERSION)); + appendControlRecord( + timestamp, + ControlRecordType.LEADER_CHANGE, + MessageUtil.toByteBuffer(leaderChangeMessage, ControlRecordUtils.LEADER_CHANGE_CURRENT_VERSION) + ); } public void appendSnapshotHeaderMessage(long timestamp, SnapshotHeaderRecord snapshotHeaderRecord) { - appendControlRecord(timestamp, ControlRecordType.SNAPSHOT_HEADER, - MessageUtil.toByteBuffer(snapshotHeaderRecord, ControlRecordUtils.SNAPSHOT_HEADER_HIGHEST_VERSION)); + appendControlRecord( + timestamp, + ControlRecordType.SNAPSHOT_HEADER, + MessageUtil.toByteBuffer(snapshotHeaderRecord, ControlRecordUtils.SNAPSHOT_HEADER_CURRENT_VERSION) + ); } public void appendSnapshotFooterMessage(long timestamp, SnapshotFooterRecord snapshotHeaderRecord) { - appendControlRecord(timestamp, ControlRecordType.SNAPSHOT_FOOTER, - MessageUtil.toByteBuffer(snapshotHeaderRecord, ControlRecordUtils.SNAPSHOT_FOOTER_HIGHEST_VERSION)); + appendControlRecord( + timestamp, + ControlRecordType.SNAPSHOT_FOOTER, + MessageUtil.toByteBuffer(snapshotHeaderRecord, ControlRecordUtils.SNAPSHOT_FOOTER_CURRENT_VERSION) + ); } /** diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index cd99f472ebb0a..7e4425d3e79c1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -266,8 +266,20 @@ public ApiKeys apiKey() { return apiKey; } + /** + * Get the throttle time in milliseconds. If the response schema does not + * support this field, then 0 will be returned. + */ public abstract int throttleTimeMs(); + /** + * Set the throttle time in the response if the schema supports it. Otherwise, + * this is a no-op. + * + * @param throttleTimeMs The throttle time in milliseconds + */ + public abstract void maybeSetThrottleTimeMs(int throttleTimeMs); + public String toString() { return data().toString(); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java index ce9a6cf7d6063..d90afd04ddcde 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java @@ -56,6 +56,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public AddOffsetsToTxnResponseData data() { return data; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java index 57b2a5a5d7c08..8038f4b8fc66d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java @@ -94,6 +94,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public Map errors() { if (cachedErrorsMap != null) { return cachedErrorsMap; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java index 5d48c39e8019d..2511e2b2db320 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java @@ -56,6 +56,15 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + + public Errors error() { + return Errors.forCode(data.errorCode()); + } + public static AllocateProducerIdsResponse parse(ByteBuffer buffer, short version) { return new AllocateProducerIdsResponse(new AllocateProducerIdsResponseData( new ByteBufferAccessor(buffer), version)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasResponse.java index fcacc5d95ef07..fc56db7e73658 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasResponse.java @@ -67,6 +67,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { Map counts = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java index 1115f06ee80a9..1668c2446bc77 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java @@ -55,6 +55,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public AlterConfigsResponseData data() { return data; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionReassignmentsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionReassignmentsResponse.java index ab166b812718c..7d6c340fd147b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionReassignmentsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionReassignmentsResponse.java @@ -55,6 +55,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { Map counts = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java index d2ace4112f4c1..9ee92f7b809cd 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java @@ -55,6 +55,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public static AlterPartitionResponse parse(ByteBuffer buffer, short version) { return new AlterPartitionResponse(new AlterPartitionResponseData(new ByteBufferAccessor(buffer), version)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java index afa658d1e150f..0c38a83ee3d7e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java @@ -53,6 +53,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { Map errorCounts = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsResponse.java index 97c0b7d17b204..86c9b006a2ce0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterUserScramCredentialsResponse.java @@ -48,6 +48,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { return errorCounts(data.results().stream().map(r -> Errors.forCode(r.errorCode()))); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiError.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiError.java index 01966532d62ff..dd127fc7a5885 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiError.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiError.java @@ -21,8 +21,6 @@ import org.apache.kafka.common.protocol.Errors; import java.util.Objects; -import java.util.concurrent.CompletionException; -import java.util.concurrent.ExecutionException; /** * Encapsulates an error code (via the Errors enum) and an optional message. Generally, the optional message is only @@ -38,15 +36,9 @@ public class ApiError { private final String message; public static ApiError fromThrowable(Throwable t) { - Throwable throwableToBeEncoded = t; - // Get the underlying cause for common exception types from the concurrent library. - // This is useful to handle cases where exceptions may be raised from a future or a - // completion stage (as might be the case for requests sent to the controller in `ControllerApis`) - if (t instanceof CompletionException || t instanceof ExecutionException) { - throwableToBeEncoded = t.getCause(); - } // Avoid populating the error message if it's a generic one. Also don't populate error // message for UNKNOWN_SERVER_ERROR to ensure we don't leak sensitive information. + Throwable throwableToBeEncoded = Errors.maybeUnwrapException(t); Errors error = Errors.forException(throwableToBeEncoded); String message = error == Errors.UNKNOWN_SERVER_ERROR || error.message().equals(throwableToBeEncoded.getMessage()) ? null : throwableToBeEncoded.getMessage(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java index 7c98eb2679b5a..a903e50b15d9e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java @@ -74,6 +74,11 @@ public int throttleTimeMs() { return this.data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public boolean shouldClientThrottle(short version) { return version >= 2; @@ -107,17 +112,25 @@ public static ApiVersionsResponse defaultApiVersionsResponse( int throttleTimeMs, ApiMessageType.ListenerType listenerType ) { - return createApiVersionsResponse(throttleTimeMs, filterApis(RecordVersion.current(), listenerType)); + return createApiVersionsResponse(throttleTimeMs, filterApis(RecordVersion.current(), listenerType), Features.emptySupportedFeatures()); } public static ApiVersionsResponse createApiVersionsResponse( int throttleTimeMs, ApiVersionCollection apiVersions + ) { + return createApiVersionsResponse(throttleTimeMs, apiVersions, Features.emptySupportedFeatures()); + } + + public static ApiVersionsResponse createApiVersionsResponse( + int throttleTimeMs, + ApiVersionCollection apiVersions, + Features latestSupportedFeatures ) { return createApiVersionsResponse( throttleTimeMs, apiVersions, - Features.emptySupportedFeatures(), + latestSupportedFeatures, Collections.emptyMap(), UNKNOWN_FINALIZED_FEATURES_EPOCH); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java index c8c0328c93a15..5ae975acd8a05 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java @@ -95,6 +95,11 @@ public int throttleTimeMs() { return DEFAULT_THROTTLE_TIME; } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + // Not supported by the response schema + } + public static BeginQuorumEpochResponse parse(ByteBuffer buffer, short version) { return new BeginQuorumEpochResponse(new BeginQuorumEpochResponseData(new ByteBufferAccessor(buffer), version)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BrokerHeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/BrokerHeartbeatResponse.java index e7d01e53c67ba..4c8b3aafc4dd2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/BrokerHeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/BrokerHeartbeatResponse.java @@ -44,6 +44,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { Map errorCounts = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationResponse.java index 8296d7a4c35cb..8b6121c376339 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationResponse.java @@ -44,6 +44,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { Map errorCounts = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java index 73b6a50268379..bc5aa0ba35a33 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownResponse.java @@ -58,6 +58,11 @@ public int throttleTimeMs() { return DEFAULT_THROTTLE_TIME; } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + // Not supported by the response schema + } + public static ControlledShutdownResponse parse(ByteBuffer buffer, short version) { return new ControlledShutdownResponse(new ControlledShutdownResponseData(new ByteBufferAccessor(buffer), version)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java index 8bc6643f9de01..cef7b73ac27e9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java @@ -43,6 +43,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public List results() { return data.results(); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java index 22c2e1259019b..0a9f9a8991bdc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java @@ -86,6 +86,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public Errors error() { return Errors.forCode(data.errorCode()); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsResponse.java index e59ac981f112a..2dcd2b200cadd 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsResponse.java @@ -62,4 +62,9 @@ public boolean shouldClientThrottle(short version) { public int throttleTimeMs() { return data.throttleTimeMs(); } + + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java index dd0627742587c..da011e224ed08 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java @@ -60,6 +60,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { HashMap counts = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java index 7482953a00d64..d0b596ed91ab1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java @@ -60,6 +60,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public List filterResults() { return data.filterResults(); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java index 4cbffda422138..3bbb08d59fb9b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java @@ -85,6 +85,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public boolean shouldClientThrottle(short version) { return version >= 1; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java index b090543faddfb..5084681f5373f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java @@ -56,6 +56,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { Map errorCounts = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java index 2090c4fd2e2ee..65a54481ba07e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsResponse.java @@ -50,6 +50,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public DeleteTopicsResponseData data() { return data; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java index c4190e65640ea..c602dab9951fc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java @@ -71,6 +71,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public ApiError error() { return new ApiError(Errors.forCode(data.errorCode()), data.errorMessage()); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java index 147414333ea08..3a052c9fe8eba 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java @@ -70,6 +70,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public DescribeClientQuotasResponseData data() { return data; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeClusterResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeClusterResponse.java index 60d931196a659..fb48476e2467d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeClusterResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeClusterResponse.java @@ -52,6 +52,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public DescribeClusterResponseData data() { return data; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java index aa7a713e8ab4e..6fd5320d8820e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java @@ -255,6 +255,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { Map errorCounts = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java index 4fd1d99652661..a922f056a89aa 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java @@ -96,6 +96,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public Errors error() { return Errors.forCode(data.errorCode()); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java index 360caf01e468e..119bedfdb19dc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java @@ -115,6 +115,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public static final String UNKNOWN_STATE = ""; public static final String UNKNOWN_PROTOCOL_TYPE = ""; public static final String UNKNOWN_PROTOCOL = ""; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java index fe8aebbc4f6b8..cbf3054217363 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java @@ -50,6 +50,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { Map errorCounts = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java index 74e9437472bc9..065a101bed6e8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java @@ -66,4 +66,9 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java index cbf945b70409a..39e050c94052b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.DescribeQuorumResponseData; -import org.apache.kafka.common.message.DescribeQuorumResponseData.ReplicaState; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; @@ -26,7 +25,6 @@ import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; /** @@ -37,7 +35,7 @@ * - {@link Errors#BROKER_NOT_AVAILABLE} * * Partition level errors: - * - {@link Errors#INVALID_REQUEST} + * - {@link Errors#NOT_LEADER_OR_FOLLOWER} * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} */ public class DescribeQuorumResponse extends AbstractResponse { @@ -72,22 +70,33 @@ public int throttleTimeMs() { return DEFAULT_THROTTLE_TIME; } - public static DescribeQuorumResponseData singletonResponse(TopicPartition topicPartition, - int leaderId, - int leaderEpoch, - long highWatermark, - List voterStates, - List observerStates) { + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + // Not supported by the response schema + } + + public static DescribeQuorumResponseData singletonErrorResponse( + TopicPartition topicPartition, + Errors error + ) { return new DescribeQuorumResponseData() .setTopics(Collections.singletonList(new DescribeQuorumResponseData.TopicData() .setTopicName(topicPartition.topic()) .setPartitions(Collections.singletonList(new DescribeQuorumResponseData.PartitionData() - .setErrorCode(Errors.NONE.code()) - .setLeaderId(leaderId) - .setLeaderEpoch(leaderEpoch) - .setHighWatermark(highWatermark) - .setCurrentVoters(voterStates) - .setObservers(observerStates))))); + .setPartitionIndex(topicPartition.partition()) + .setErrorCode(error.code()))))); + } + + + public static DescribeQuorumResponseData singletonResponse( + TopicPartition topicPartition, + DescribeQuorumResponseData.PartitionData partitionData + ) { + return new DescribeQuorumResponseData() + .setTopics(Collections.singletonList(new DescribeQuorumResponseData.TopicData() + .setTopicName(topicPartition.topic()) + .setPartitions(Collections.singletonList(partitionData + .setPartitionIndex(topicPartition.partition()))))); } public static DescribeQuorumResponse parse(ByteBuffer buffer, short version) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java index cf151b35bba35..5eef63b0ce4bb 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeTransactionsResponse.java @@ -64,4 +64,10 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + } + diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsResponse.java index 001cefae41a6b..58ba4212949c6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsResponse.java @@ -48,6 +48,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { return errorCounts(data.results().stream().map(r -> Errors.forCode(r.errorCode()))); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersResponse.java index 88d4d19fc021e..2e82cd4c9a5ef 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersResponse.java @@ -62,6 +62,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { HashMap counts = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java index ac2c0c5c9d5c3..b3a236adc69cb 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java @@ -73,6 +73,11 @@ public int throttleTimeMs() { return DEFAULT_THROTTLE_TIME; } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + // Not supported by the response schema + } + public static EndQuorumEpochResponseData singletonResponse( Errors topLevelError, TopicPartition topicPartition, diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java index 029e7d0ce5909..0ab01bb1a3d33 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java @@ -50,6 +50,10 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } public Errors error() { return Errors.forCode(data.errorCode()); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeResponse.java index 529f616bb26fc..4f534b6721f4e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeResponse.java @@ -67,6 +67,11 @@ public int throttleTimeMs() { return DEFAULT_THROTTLE_TIME; } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + // Not supported by the response schema + } + public static EnvelopeResponse parse(ByteBuffer buffer, short version) { return new EnvelopeResponse(new EnvelopeResponseData(new ByteBufferAccessor(buffer), version)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenResponse.java index 163ee78d0adcd..ec43f3371b569 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenResponse.java @@ -56,6 +56,11 @@ public ExpireDelegationTokenResponseData data() { return data; } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public int throttleTimeMs() { return data.throttleTimeMs(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index a4af4ca2a2370..cd177945830d3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -128,6 +128,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public int sessionId() { return data.sessionId(); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchSnapshotResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchSnapshotResponse.java index 7c1ce27f3da8e..d9abff66fe96e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchSnapshotResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchSnapshotResponse.java @@ -61,6 +61,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public FetchSnapshotResponseData data() { return data; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java index 080ba24c3bd9d..e96e8a0c0db9a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java @@ -64,6 +64,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public boolean hasError() { return error() != Errors.NONE; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java index eb402fcbab9f7..aebb903e967e7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -48,6 +48,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public Errors error() { return Errors.forCode(data.errorCode()); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsResponse.java index b5887de9b4b75..826be30a8d3fc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsResponse.java @@ -90,6 +90,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public static IncrementalAlterConfigsResponse parse(ByteBuffer buffer, short version) { return new IncrementalAlterConfigsResponse(new IncrementalAlterConfigsResponseData( new ByteBufferAccessor(buffer), version)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java index f8451d7863b3f..96c7a4d400ced 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java @@ -48,6 +48,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { return errorCounts(Errors.forCode(data.errorCode())); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index 220a59d183428..774506357bbb9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -70,6 +70,20 @@ public static void validateGroupInstanceId(String id) { }); } + /** + * Ensures that the provided {@code reason} remains within a range of 255 chars. + * @param reason This is the reason that is sent to the broker over the wire + * as a part of {@code JoinGroupRequest} or {@code LeaveGroupRequest} messages. + * @return a provided reason as is or truncated reason if it exceeds the 255 chars threshold. + */ + public static String maybeTruncateReason(final String reason) { + if (reason.length() > 255) { + return reason.substring(0, 255); + } else { + return reason; + } + } + public JoinGroupRequest(JoinGroupRequestData data, short version) { super(ApiKeys.JOIN_GROUP, version); this.data = data; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java index 336c82462a21d..5a4332efde9a9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java @@ -47,6 +47,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public Errors error() { return Errors.forCode(data.errorCode()); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java index c7c04e2d99b41..0d40581d68e8a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java @@ -99,6 +99,11 @@ public int throttleTimeMs() { return DEFAULT_THROTTLE_TIME; } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + // Not supported by the response schema + } + public static LeaderAndIsrResponse parse(ByteBuffer buffer, short version) { return new LeaderAndIsrResponse(new LeaderAndIsrResponseData(new ByteBufferAccessor(buffer), version), version); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java index 9a59139f4e77c..d39766be68f82 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java @@ -82,6 +82,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public List memberResponses() { return data.members(); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java index 270c43c0568ad..a12f85341d6a4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java @@ -43,6 +43,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { return errorCounts(Errors.forCode(data.errorCode())); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsResponse.java index 8c4a51b542b47..53356dd93be0a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsResponse.java @@ -64,6 +64,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public ListOffsetsResponseData data() { return data; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsResponse.java index 4a890e8b50cd1..cbf06d4c46624 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsResponse.java @@ -53,6 +53,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { return errorCounts(Errors.forCode(data.errorCode())); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java index 13ed184fc3408..f509543025b92 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java @@ -59,4 +59,9 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index 3696b047abad1..47cdd3f0d7e90 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -84,6 +84,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + /** * Get a map of the topics which had metadata errors * @return the map diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 2ed0e312983ce..713b68974a11d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -107,6 +107,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public boolean shouldClientThrottle(short version) { return version >= 4; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java index 79f6f4e6d3495..993a589af69fe 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java @@ -77,6 +77,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public boolean shouldClientThrottle(short version) { return version >= 0; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 213182ec8c4a5..2d585a582ae87 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -173,8 +173,8 @@ public OffsetFetchResponse(int throttleTimeMs, Errors error, Map errors, Map> responseData) { + Map errors, + Map> responseData) { super(ApiKeys.OFFSET_FETCH); List groupList = new ArrayList<>(); for (Entry> entry : responseData.entrySet()) { @@ -245,12 +245,21 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public boolean hasError() { return error != Errors.NONE; } public boolean groupHasError(String groupId) { - return groupLevelErrors.get(groupId) != Errors.NONE; + Errors error = groupLevelErrors.get(groupId); + if (error == null) { + return this.error != null && this.error != Errors.NONE; + } + return error != Errors.NONE; } public Errors error() { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java index 893d5a2af20a0..10c257c0a37cf 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java @@ -68,6 +68,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public static OffsetsForLeaderEpochResponse parse(ByteBuffer buffer, short version) { return new OffsetsForLeaderEpochResponse(new OffsetForLeaderEpochResponseData(new ByteBufferAccessor(buffer), version)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 7c9d70b8d6784..a00fdecc58645 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -116,6 +116,11 @@ public int throttleTimeMs() { return this.data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { Map errorCounts = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenResponse.java index 30708ff038c25..8ea85d74db6c7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenResponse.java @@ -53,6 +53,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public Errors error() { return Errors.forCode(data.errorCode()); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java index bd12d3d4ae7bb..d6ca8c170dc45 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java @@ -67,6 +67,11 @@ public int throttleTimeMs() { return DEFAULT_THROTTLE_TIME; } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + // Not supported by the response schema + } + @Override public SaslAuthenticateResponseData data() { return data; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java index 63c047a06196b..5097711e73787 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java @@ -57,6 +57,11 @@ public int throttleTimeMs() { return DEFAULT_THROTTLE_TIME; } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + // Not supported by the response schema + } + @Override public SaslHandshakeResponseData data() { return data; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java index 10ab153f440b9..cb66f4915d168 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaResponse.java @@ -70,6 +70,11 @@ public int throttleTimeMs() { return DEFAULT_THROTTLE_TIME; } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + // Not supported by the response schema + } + @Override public StopReplicaResponseData data() { return data; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java index 822a3e78b9949..596110242902c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java @@ -38,6 +38,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + public Errors error() { return Errors.forCode(data.errorCode()); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java index b4de54741e6d0..18244fcb17c33 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java @@ -87,6 +87,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { return errorCounts(data.topics().stream().flatMap(topic -> diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UnregisterBrokerResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UnregisterBrokerResponse.java index b508ac3ef9e54..623e6f28076fa 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UnregisterBrokerResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UnregisterBrokerResponse.java @@ -44,6 +44,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public Map errorCounts() { Map errorCounts = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java index 26825a0c24763..567464d85dbae 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java @@ -63,6 +63,11 @@ public int throttleTimeMs() { return data.throttleTimeMs(); } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + @Override public String toString() { return data.toString(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java index cc7749a47242c..d5960d7cbb923 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataResponse.java @@ -47,6 +47,11 @@ public int throttleTimeMs() { return DEFAULT_THROTTLE_TIME; } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + // Not supported by the response schema + } + public static UpdateMetadataResponse parse(ByteBuffer buffer, short version) { return new UpdateMetadataResponse(new UpdateMetadataResponseData(new ByteBufferAccessor(buffer), version)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java index 51991adcf0cbb..f79c6eeb0de19 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java @@ -92,6 +92,11 @@ public int throttleTimeMs() { return DEFAULT_THROTTLE_TIME; } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + // Not supported by the response schema + } + public static VoteResponse parse(ByteBuffer buffer, short version) { return new VoteResponse(new VoteResponseData(new ByteBufferAccessor(buffer), version)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java index fd2a834d24569..a7d22e4493e67 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java @@ -108,6 +108,11 @@ public int throttleTimeMs() { return DEFAULT_THROTTLE_TIME; } + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + // Not supported by the response schema + } + @Override public Map errorCounts() { Map errorCounts = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensions.java b/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensions.java index c129f1ec400f7..ca4c4df6079ba 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensions.java +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensions.java @@ -19,15 +19,34 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Set; +import java.util.StringJoiner; +import javax.security.auth.Subject; /** - * A simple immutable value object class holding customizable SASL extensions + * A simple immutable value object class holding customizable SASL extensions. + * + *

+ * + * Note on object identity and equality: SaslExtensions intentionally + * overrides the standard {@link #equals(Object)} and {@link #hashCode()} methods calling their + * respective {@link Object#equals(Object)} and {@link Object#hashCode()} implementations. In so + * doing, it provides equality only via reference identity and will not base equality on + * the underlying values of its {@link #extensionsMap extentions map}. + * + *

+ * + * The reason for this approach to equality is based off of the manner in which + * credentials are stored in a {@link Subject}. SaslExtensions are added to and + * removed from a {@link Subject} via its {@link Subject#getPublicCredentials() public credentials}. + * The public credentials are stored in a {@link Set} in the {@link Subject}, so object equality + * therefore becomes a concern. With shallow, reference-based equality, distinct + * SaslExtensions instances with the same map values can be considered unique. This is + * critical to operations like token refresh. + * + * See KAFKA-14062 for more detail. */ public class SaslExtensions { - /** - * An "empty" instance indicating no SASL extensions - */ - public static final SaslExtensions NO_SASL_EXTENSIONS = new SaslExtensions(Collections.emptyMap()); private final Map extensionsMap; public SaslExtensions(Map extensionsMap) { @@ -41,21 +60,59 @@ public Map map() { return extensionsMap; } + /** + * Creates an "empty" instance indicating no SASL extensions. Do not cache the result of + * this method call for use by multiple {@link Subject}s as the references need to be + * unique. + * + *

+ * + * See the class-level documentation for details. + * @return Unique, but empty, SaslExtensions instance + */ + @SuppressWarnings("unchecked") + public static SaslExtensions empty() { + // It's ok to re-use the EMPTY_MAP instance as the object equality is on the outer + // SaslExtensions reference. + return new SaslExtensions(Collections.EMPTY_MAP); + } + + /** + * Implements equals using the reference comparison implementation from + * {@link Object#equals(Object)}. + * + *

+ * + * See the class-level documentation for details. + * + * @param o Other object to compare + * @return True if o == this + */ @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - return extensionsMap.equals(((SaslExtensions) o).extensionsMap); + public final boolean equals(Object o) { + return super.equals(o); } + /** + * Implements hashCode using the native implementation from + * {@link Object#hashCode()}. + * + *

+ * + * See the class-level documentation for details. + * + * @return Hash code of instance + */ @Override - public String toString() { - return extensionsMap.toString(); + public final int hashCode() { + return super.hashCode(); } @Override - public int hashCode() { - return extensionsMap.hashCode(); + public String toString() { + return new StringJoiner(", ", SaslExtensions.class.getSimpleName() + "[", "]") + .add("extensionsMap=" + extensionsMap) + .toString(); } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensionsCallback.java b/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensionsCallback.java index c5bd449e0cc08..f2010afda67e4 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensionsCallback.java +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/SaslExtensionsCallback.java @@ -26,13 +26,13 @@ * in the SASL exchange. */ public class SaslExtensionsCallback implements Callback { - private SaslExtensions extensions = SaslExtensions.NO_SASL_EXTENSIONS; + private SaslExtensions extensions = SaslExtensions.empty(); /** * Returns always non-null {@link SaslExtensions} consisting of the extension * names and values that are sent by the client to the server in the initial * client SASL authentication message. The default value is - * {@link SaslExtensions#NO_SASL_EXTENSIONS} so that if this callback is + * {@link SaslExtensions#empty()} so that if this callback is * unhandled the client will see a non-null value. */ public SaslExtensions extensions() { diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java index 019723b6b408d..51b02952a6239 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java @@ -27,6 +27,7 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.SaslAuthenticateResponseData; import org.apache.kafka.common.message.SaslHandshakeResponseData; +import org.apache.kafka.common.network.InvalidReceiveException; import org.apache.kafka.common.network.Authenticator; import org.apache.kafka.common.network.ByteBufferSend; import org.apache.kafka.common.network.ChannelBuilders; @@ -88,8 +89,6 @@ import java.util.function.Supplier; public class SaslServerAuthenticator implements Authenticator { - // GSSAPI limits requests to 64K, but we allow a bit extra for custom SASL mechanisms - static final int MAX_RECEIVE_SIZE = 524288; private static final Logger LOG = LoggerFactory.getLogger(SaslServerAuthenticator.class); /** @@ -140,6 +139,7 @@ private enum SaslState { private String saslMechanism; // buffers used in `authenticate` + private Integer saslAuthRequestMaxReceiveSize; private NetworkReceive netInBuffer; private Send netOutBuffer; private Send authenticationFailureSend = null; @@ -189,6 +189,10 @@ public SaslServerAuthenticator(Map configs, // Note that the old principal builder does not support SASL, so we do not need to pass the // authenticator or the transport layer this.principalBuilder = ChannelBuilders.createPrincipalBuilder(configs, kerberosNameParser, null); + + saslAuthRequestMaxReceiveSize = (Integer) configs.get(BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG); + if (saslAuthRequestMaxReceiveSize == null) + saslAuthRequestMaxReceiveSize = BrokerSecurityConfigs.DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE; } private void createSaslServer(String mechanism) throws IOException { @@ -252,9 +256,13 @@ public void authenticate() throws IOException { } // allocate on heap (as opposed to any socket server memory pool) - if (netInBuffer == null) netInBuffer = new NetworkReceive(MAX_RECEIVE_SIZE, connectionId); + if (netInBuffer == null) netInBuffer = new NetworkReceive(saslAuthRequestMaxReceiveSize, connectionId); - netInBuffer.readFrom(transportLayer); + try { + netInBuffer.readFrom(transportLayer); + } catch (InvalidReceiveException e) { + throw new SaslAuthenticationException("Failing SASL authentication due to invalid receive size", e); + } if (!netInBuffer.complete()) return; netInBuffer.payload().rewind(); 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 a356f0da3ddb9..52623ff9fd4f2 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 @@ -108,7 +108,7 @@ public OAuthBearerClientInitialResponse(String tokenValue, String authorizationI this.tokenValue = Objects.requireNonNull(tokenValue, "token value must not be null"); this.authorizationId = authorizationId == null ? "" : authorizationId; validateExtensions(extensions); - this.saslExtensions = extensions != null ? extensions : SaslExtensions.NO_SASL_EXTENSIONS; + this.saslExtensions = extensions != null ? extensions : SaslExtensions.empty(); } /** diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java index b52952a16f1d1..780d97e9decf6 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetriever.java @@ -240,6 +240,7 @@ static String handleOutput(final HttpURLConnection con) throws IOException { log.debug("handleOutput - responseCode: {}", responseCode); String responseBody = null; + String errorResponseBody = null; try (InputStream is = con.getInputStream()) { ByteArrayOutputStream os = new ByteArrayOutputStream(); @@ -247,27 +248,41 @@ static String handleOutput(final HttpURLConnection con) throws IOException { copy(is, os); responseBody = os.toString(StandardCharsets.UTF_8.name()); } catch (Exception e) { + // there still can be useful error response from the servers, lets get it + try (InputStream is = con.getErrorStream()) { + ByteArrayOutputStream os = new ByteArrayOutputStream(); + log.debug("handleOutput - preparing to read error response body from {}", con.getURL()); + copy(is, os); + errorResponseBody = os.toString(StandardCharsets.UTF_8.name()); + } catch (Exception e2) { + log.warn("handleOutput - error retrieving error information", e2); + } log.warn("handleOutput - error retrieving data", e); } if (responseCode == HttpURLConnection.HTTP_OK || responseCode == HttpURLConnection.HTTP_CREATED) { - log.debug("handleOutput - responseCode: {}, response: {}", responseCode, responseBody); + log.debug("handleOutput - responseCode: {}, response: {}, error response: {}", responseCode, responseBody, + errorResponseBody); if (responseBody == null || responseBody.isEmpty()) - throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s", responseCode, con.getURL())); + throw new IOException(String.format("The token endpoint response was unexpectedly empty despite response code %s from %s and error message %s", + responseCode, con.getURL(), formatErrorMessage(errorResponseBody))); return responseBody; } else { - log.warn("handleOutput - error response code: {}, error response body: {}", responseCode, responseBody); + log.warn("handleOutput - error response code: {}, response body: {}, error response body: {}", responseCode, + responseBody, errorResponseBody); if (UNRETRYABLE_HTTP_CODES.contains(responseCode)) { // We know that this is a non-transient error, so let's not keep retrying the // request unnecessarily. - throw new UnretryableException(new IOException(String.format("The response code %s was encountered reading the token endpoint response; will not attempt further retries", responseCode))); + throw new UnretryableException(new IOException(String.format("The response code %s and error response %s was encountered reading the token endpoint response; will not attempt further retries", + responseCode, formatErrorMessage(errorResponseBody)))); } else { // We don't know if this is a transient (retryable) error or not, so let's assume // it is. - throw new IOException(String.format("The unexpected response code %s was encountered reading the token endpoint response", responseCode)); + throw new IOException(String.format("The unexpected response code %s and error message %s was encountered reading the token endpoint response", + responseCode, formatErrorMessage(errorResponseBody))); } } } @@ -280,6 +295,26 @@ static void copy(InputStream is, OutputStream os) throws IOException { os.write(buf, 0, b); } + static String formatErrorMessage(String errorResponseBody) { + if (errorResponseBody == null || errorResponseBody.trim().equals("")) { + return "{}"; + } + ObjectMapper mapper = new ObjectMapper(); + try { + JsonNode rootNode = mapper.readTree(errorResponseBody); + if (!rootNode.at("/error").isMissingNode()) { + return String.format("{%s - %s}", rootNode.at("/error"), rootNode.at("/error_description")); + } else if (!rootNode.at("/errorCode").isMissingNode()) { + return String.format("{%s - %s}", rootNode.at("/errorCode"), rootNode.at("/errorSummary")); + } else { + return errorResponseBody; + } + } catch (Exception e) { + log.warn("Error parsing error response", e); + } + return String.format("{%s}", errorResponseBody); + } + static String parseAccessToken(String responseBody) throws IOException { log.debug("parseAccessToken - responseBody: {}", responseBody); ObjectMapper mapper = new ObjectMapper(); @@ -308,7 +343,8 @@ static String formatAuthorizationHeader(String clientId, String clientSecret) { clientSecret = sanitizeString("the token endpoint request client secret parameter", clientSecret); String s = String.format("%s:%s", clientId, clientSecret); - String encoded = Base64.getUrlEncoder().encodeToString(Utils.utf8(s)); + // Per RFC-7617, we need to use the *non-URL safe* base64 encoder. See KAFKA-14496. + String encoded = Base64.getEncoder().encodeToString(Utils.utf8(s)); return String.format("Basic %s", encoded); } diff --git a/clients/src/main/java/org/apache/kafka/server/quota/ClientQuotaCallback.java b/clients/src/main/java/org/apache/kafka/server/quota/ClientQuotaCallback.java index 210e9f45840e2..bb1a5deafe02a 100644 --- a/clients/src/main/java/org/apache/kafka/server/quota/ClientQuotaCallback.java +++ b/clients/src/main/java/org/apache/kafka/server/quota/ClientQuotaCallback.java @@ -55,7 +55,7 @@ public interface ClientQuotaCallback extends Configurable { /** * Quota configuration update callback that is invoked when quota configuration for an entity is - * updated in ZooKeeper. This is useful to track configured quotas if built-in quota configuration + * updated in the quorum. This is useful to track configured quotas if built-in quota configuration * tools are used for quota management. * * @param quotaType Type of quota being updated @@ -66,7 +66,7 @@ public interface ClientQuotaCallback extends Configurable { /** * Quota configuration removal callback that is invoked when quota configuration for an entity is - * removed in ZooKeeper. This is useful to track configured quotas if built-in quota configuration + * removed in the quorum. This is useful to track configured quotas if built-in quota configuration * tools are used for quota management. * * @param quotaType Type of quota being updated diff --git a/clients/src/main/resources/common/message/JoinGroupResponse.json b/clients/src/main/resources/common/message/JoinGroupResponse.json index 79d596df27054..d01c2c1c02879 100644 --- a/clients/src/main/resources/common/message/JoinGroupResponse.json +++ b/clients/src/main/resources/common/message/JoinGroupResponse.json @@ -58,7 +58,7 @@ { "name": "Members", "type": "[]JoinGroupResponseMember", "versions": "0+", "fields": [ { "name": "MemberId", "type": "string", "versions": "0+", "about": "The group member ID." }, - { "name": "GroupInstanceId", "type": "string", "versions": "5+", + { "name": "GroupInstanceId", "type": "string", "versions": "5+", "ignorable": true, "nullableVersions": "5+", "default": "null", "about": "The unique identifier of the consumer instance provided by end user." }, { "name": "Metadata", "type": "bytes", "versions": "0+", diff --git a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java index a4383d8991e6a..30e7c3ab18635 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java @@ -791,6 +791,38 @@ public void testNodeIfOffline() { assertEquals(metadata.fetch().nodeById(1).id(), 1); } + @Test + public void testNodeIfOnlineWhenNotInReplicaSet() { + Map partitionCounts = new HashMap<>(); + partitionCounts.put("topic-1", 1); + Node node0 = new Node(0, "localhost", 9092); + + MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 2, Collections.emptyMap(), partitionCounts, _tp -> 99, + (error, partition, leader, leaderEpoch, replicas, isr, offlineReplicas) -> + new MetadataResponse.PartitionMetadata(error, partition, Optional.of(node0.id()), leaderEpoch, + Collections.singletonList(node0.id()), Collections.emptyList(), + Collections.emptyList()), ApiKeys.METADATA.latestVersion(), Collections.emptyMap()); + metadata.updateWithCurrentRequestVersion(emptyMetadataResponse(), false, 0L); + metadata.updateWithCurrentRequestVersion(metadataResponse, false, 10L); + + TopicPartition tp = new TopicPartition("topic-1", 0); + + assertEquals(1, metadata.fetch().nodeById(1).id()); + assertFalse(metadata.fetch().nodeIfOnline(tp, 1).isPresent()); + } + + @Test + public void testNodeIfOnlineNonExistentTopicPartition() { + MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(2, Collections.emptyMap()); + metadata.updateWithCurrentRequestVersion(metadataResponse, false, 0L); + + TopicPartition tp = new TopicPartition("topic-1", 0); + + assertEquals(metadata.fetch().nodeById(0).id(), 0); + assertNull(metadata.fetch().partition(tp)); + assertEquals(metadata.fetch().nodeIfOnline(tp, 0), Optional.empty()); + } + @Test public void testLeaderMetadataInconsistentWithBrokerMetadata() { // Tests a reordering scenario which can lead to inconsistent leader state. diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java index 6f98a166b17ca..d8b9f427d6b24 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java @@ -24,6 +24,7 @@ import org.apache.kafka.clients.HostResolver; import org.apache.kafka.clients.admin.CreateTopicsResult.TopicMetadataAndConfig; import org.apache.kafka.clients.admin.internals.MetadataOperationContext; +import org.apache.kafka.clients.admin.internals.CoordinatorKey; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaFuture; @@ -104,14 +105,17 @@ public static DescribeTopicsResult describeTopicsResult(Map KafkaFuture.completedFuture(e.getValue())))); } - public static ListConsumerGroupOffsetsResult listConsumerGroupOffsetsResult(Map offsets) { - return new ListConsumerGroupOffsetsResult(KafkaFuture.completedFuture(offsets)); + public static ListConsumerGroupOffsetsResult listConsumerGroupOffsetsResult(Map> offsets) { + Map>> resultMap = offsets.entrySet().stream() + .collect(Collectors.toMap(e -> CoordinatorKey.byGroupId(e.getKey()), + e -> KafkaFutureImpl.completedFuture(e.getValue()))); + return new ListConsumerGroupOffsetsResult(resultMap); } - public static ListConsumerGroupOffsetsResult listConsumerGroupOffsetsResult(KafkaException exception) { + public static ListConsumerGroupOffsetsResult listConsumerGroupOffsetsResult(String group, KafkaException exception) { final KafkaFutureImpl> future = new KafkaFutureImpl<>(); future.completeExceptionally(exception); - return new ListConsumerGroupOffsetsResult(future); + return new ListConsumerGroupOffsetsResult(Collections.singletonMap(CoordinatorKey.byGroupId(group), future)); } /** 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 523a961f96493..652c29a6e4b11 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 @@ -108,6 +108,7 @@ import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData.CredentialInfo; import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult; import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult; +import org.apache.kafka.common.message.FindCoordinatorRequestData; import org.apache.kafka.common.message.FindCoordinatorResponseData; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.AlterConfigsResourceResponse; @@ -130,6 +131,8 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection; import org.apache.kafka.common.message.OffsetFetchRequestData; +import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestGroup; +import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestTopics; import org.apache.kafka.common.message.UnregisterBrokerResponseData; import org.apache.kafka.common.message.WriteTxnMarkersResponseData; import org.apache.kafka.common.protocol.ApiKeys; @@ -192,6 +195,7 @@ import org.apache.kafka.common.requests.OffsetDeleteResponse; import org.apache.kafka.common.requests.OffsetFetchRequest; import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.requests.UnregisterBrokerResponse; import org.apache.kafka.common.requests.UpdateFeaturesRequest; @@ -224,6 +228,7 @@ 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.OptionalLong; @@ -266,6 +271,7 @@ public class KafkaAdminClientTest { private static final Logger log = LoggerFactory.getLogger(KafkaAdminClientTest.class); private static final String GROUP_ID = "group-0"; + private static final int THROTTLE = 10; @Test public void testDefaultApiTimeoutAndRequestTimeoutConflicts() { @@ -501,6 +507,21 @@ private static FindCoordinatorResponse prepareOldFindCoordinatorResponse(Errors return FindCoordinatorResponse.prepareOldResponse(error, node); } + private static FindCoordinatorResponse prepareBatchedFindCoordinatorResponse(Errors error, Node node, Collection groups) { + FindCoordinatorResponseData data = new FindCoordinatorResponseData(); + List coordinators = groups.stream() + .map(group -> new FindCoordinatorResponseData.Coordinator() + .setErrorCode(error.code()) + .setErrorMessage(error.message()) + .setKey(group) + .setHost(node.host()) + .setPort(node.port()) + .setNodeId(node.id())) + .collect(Collectors.toList()); + data.setCoordinators(coordinators); + return new FindCoordinatorResponse(data); + } + private static MetadataResponse prepareMetadataResponse(Cluster cluster, Errors error) { List metadata = new ArrayList<>(); for (String topic : cluster.topics()) { @@ -586,8 +607,8 @@ private static ApiVersionsResponse prepareApiVersionsResponseForDescribeFeatures .setErrorCode(error.code())); } - private static QuorumInfo defaultQuorumInfo(Boolean emptyOptionals) { - return new QuorumInfo(1, + private static QuorumInfo defaultQuorumInfo(boolean emptyOptionals) { + return new QuorumInfo(1, 1, 1L, singletonList(new QuorumInfo.ReplicaState(1, 100, emptyOptionals ? OptionalLong.empty() : OptionalLong.of(1000), emptyOptionals ? OptionalLong.empty() : OptionalLong.of(1000))), @@ -604,8 +625,8 @@ private static DescribeQuorumResponse prepareDescribeQuorumResponse( Boolean partitionCountError, Boolean partitionIndexError, Boolean emptyOptionals) { - String topicName = topicNameError ? "RANDOM" : Topic.METADATA_TOPIC_NAME; - Integer partitionIndex = partitionIndexError ? 1 : Topic.METADATA_TOPIC_PARTITION.partition(); + String topicName = topicNameError ? "RANDOM" : Topic.CLUSTER_METADATA_TOPIC_NAME; + Integer partitionIndex = partitionIndexError ? 1 : Topic.CLUSTER_METADATA_TOPIC_PARTITION.partition(); List topics = new ArrayList<>(); List partitions = new ArrayList<>(); for (int i = 0; i < (partitionCountError ? 2 : 1); i++) { @@ -616,8 +637,8 @@ private static DescribeQuorumResponse prepareDescribeQuorumResponse( replica.setLastCaughtUpTimestamp(emptyOptionals ? -1 : 1000); partitions.add(new DescribeQuorumResponseData.PartitionData().setPartitionIndex(partitionIndex) .setLeaderId(1) - .setLeaderEpoch(0) - .setHighWatermark(0) + .setLeaderEpoch(1) + .setHighWatermark(1) .setCurrentVoters(singletonList(replica)) .setObservers(singletonList(replica)) .setErrorCode(partitionLevelError.code())); @@ -3056,7 +3077,17 @@ public void testDescribeNonConsumerGroups() throws Exception { } @Test - public void testListConsumerGroupOffsetsOptions() throws Exception { + public void testListConsumerGroupOffsetsOptionsWithUnbatchedApi() throws Exception { + verifyListConsumerGroupOffsetsOptions(false); + } + + @Test + public void testListConsumerGroupOffsetsOptionsWithBatchedApi() throws Exception { + verifyListConsumerGroupOffsetsOptions(true); + } + + @SuppressWarnings("deprecation") + private void verifyListConsumerGroupOffsetsOptions(boolean batchedApi) throws Exception { final Cluster cluster = mockCluster(3, 0); final Time time = new MockTime(); @@ -3066,22 +3097,32 @@ public void testListConsumerGroupOffsetsOptions() throws Exception { env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - final TopicPartition tp1 = new TopicPartition("A", 0); - final ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions(); - options.topicPartitions(Collections.singletonList(tp1)).requireStable(true); - final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(GROUP_ID, options); + final List partitions = Collections.singletonList(new TopicPartition("A", 0)); + final ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions() + .requireStable(true) + .timeoutMs(300); + if (batchedApi) { + final ListConsumerGroupOffsetsSpec groupSpec = new ListConsumerGroupOffsetsSpec() + .topicPartitions(partitions); + env.adminClient().listConsumerGroupOffsets(Collections.singletonMap(GROUP_ID, groupSpec), options); + } else { + env.adminClient().listConsumerGroupOffsets(GROUP_ID, options.topicPartitions(partitions)); + } final MockClient mockClient = env.kafkaClient(); - TestUtils.waitForCondition(() -> { - final ClientRequest clientRequest = mockClient.requests().peek(); - if (clientRequest != null) { - OffsetFetchRequestData data = ((OffsetFetchRequest.Builder) clientRequest.requestBuilder()).data; - return data.requireStable() && - data.topics().get(0).name().equals("A") && - data.topics().get(0).partitionIndexes().equals(Collections.singletonList(0)); - } - return false; - }, "Failed awaiting ListConsumerGroupOffsets request"); + waitForRequest(mockClient, ApiKeys.OFFSET_FETCH); + + ClientRequest clientRequest = mockClient.requests().peek(); + assertNotNull(clientRequest); + assertEquals(300, clientRequest.requestTimeoutMs()); + OffsetFetchRequestData data = ((OffsetFetchRequest.Builder) clientRequest.requestBuilder()).data; + assertTrue(data.requireStable()); + assertEquals(Collections.singletonList(GROUP_ID), + data.groups().stream().map(OffsetFetchRequestGroup::groupId).collect(Collectors.toList())); + assertEquals(Collections.singletonList("A"), + data.groups().get(0).topics().stream().map(OffsetFetchRequestTopics::name).collect(Collectors.toList())); + assertEquals(Collections.singletonList(0), + data.groups().get(0).topics().get(0).partitionIndexes()); } } @@ -3095,12 +3136,11 @@ public void testListConsumerGroupOffsetsNumRetries() throws Exception { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(GROUP_ID); - TestUtils.assertFutureError(result.partitionsToOffsetAndMetadata(), TimeoutException.class); } } @@ -3124,16 +3164,16 @@ public void testListConsumerGroupOffsetsRetryBackoff() throws Exception { mockClient.prepareResponse(body -> { firstAttemptTime.set(time.milliseconds()); return true; - }, new OffsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); + }, offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); mockClient.prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); mockClient.prepareResponse(body -> { secondAttemptTime.set(time.milliseconds()); return true; - }, new OffsetFetchResponse(Errors.NONE, Collections.emptyMap())); + }, offsetFetchResponse(Errors.NONE, Collections.emptyMap())); - final KafkaFuture> future = env.adminClient().listConsumerGroupOffsets("group-0").partitionsToOffsetAndMetadata(); + final KafkaFuture> future = env.adminClient().listConsumerGroupOffsets(GROUP_ID).partitionsToOffsetAndMetadata(); TestUtils.waitForCondition(() -> mockClient.numAwaitingResponses() == 1, "Failed awaiting ListConsumerGroupOffsets first request failure"); TestUtils.waitForCondition(() -> ((KafkaAdminClient) env.adminClient()).numPendingCalls() == 1, "Failed to add retry ListConsumerGroupOffsets call on first failure"); @@ -3157,7 +3197,8 @@ public void testListConsumerGroupOffsetsRetriableErrors() throws Exception { prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); env.kafkaClient().prepareResponse( - new OffsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); + offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); + /* * We need to return two responses here, one for NOT_COORDINATOR call when calling list consumer offsets * api using coordinator that has moved. This will retry whole operation. So we need to again respond with a @@ -3166,19 +3207,19 @@ public void testListConsumerGroupOffsetsRetriableErrors() throws Exception { * And the same reason for the following COORDINATOR_NOT_AVAILABLE error response */ env.kafkaClient().prepareResponse( - new OffsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); + offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); env.kafkaClient().prepareResponse( prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); env.kafkaClient().prepareResponse( - new OffsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); + offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); env.kafkaClient().prepareResponse( prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); env.kafkaClient().prepareResponse( - new OffsetFetchResponse(Errors.NONE, Collections.emptyMap())); + offsetFetchResponse(Errors.NONE, Collections.emptyMap())); final ListConsumerGroupOffsetsResult errorResult1 = env.adminClient().listConsumerGroupOffsets(GROUP_ID); @@ -3199,8 +3240,7 @@ public void testListConsumerGroupOffsetsNonRetriableErrors() throws Exception { env.kafkaClient().prepareResponse( prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - env.kafkaClient().prepareResponse( - new OffsetFetchResponse(error, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(error, Collections.emptyMap())); ListConsumerGroupOffsetsResult errorResult = env.adminClient().listConsumerGroupOffsets(GROUP_ID); @@ -3220,7 +3260,7 @@ public void testListConsumerGroupOffsets() throws Exception { env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); // Retriable errors should be retried - env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); /* * We need to return two responses here, one for NOT_COORDINATOR error when calling list consumer group offsets @@ -3229,10 +3269,10 @@ public void testListConsumerGroupOffsets() throws Exception { * * And the same reason for the following COORDINATOR_NOT_AVAILABLE error response */ - env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); TopicPartition myTopicPartition0 = new TopicPartition("my_topic", 0); @@ -3249,7 +3289,7 @@ public void testListConsumerGroupOffsets() throws Exception { Optional.empty(), "", Errors.NONE)); responseData.put(myTopicPartition3, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, Optional.empty(), "", Errors.NONE)); - env.kafkaClient().prepareResponse(new OffsetFetchResponse(Errors.NONE, responseData)); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NONE, responseData)); final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(GROUP_ID); final Map partitionToOffsetAndMetadata = result.partitionsToOffsetAndMetadata().get(); @@ -3263,6 +3303,144 @@ public void testListConsumerGroupOffsets() throws Exception { } } + @Test + public void testBatchedListConsumerGroupOffsets() throws Exception { + Cluster cluster = mockCluster(1, 0); + Time time = new MockTime(); + Map groupSpecs = batchedListConsumerGroupOffsetsSpec(); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, cluster, AdminClientConfig.RETRIES_CONFIG, "0")) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareBatchedFindCoordinatorResponse(Errors.NONE, env.cluster().controller(), groupSpecs.keySet())); + + ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(groupSpecs, new ListConsumerGroupOffsetsOptions()); + sendOffsetFetchResponse(env.kafkaClient(), groupSpecs, true, Errors.NONE); + + verifyListOffsetsForMultipleGroups(groupSpecs, result); + } + } + + @Test + public void testBatchedListConsumerGroupOffsetsWithNoFindCoordinatorBatching() throws Exception { + Cluster cluster = mockCluster(1, 0); + Time time = new MockTime(); + Map groupSpecs = batchedListConsumerGroupOffsetsSpec(); + + ApiVersion findCoordinatorV3 = new ApiVersion() + .setApiKey(ApiKeys.FIND_COORDINATOR.id) + .setMinVersion((short) 0) + .setMaxVersion((short) 3); + ApiVersion offsetFetchV7 = new ApiVersion() + .setApiKey(ApiKeys.OFFSET_FETCH.id) + .setMinVersion((short) 0) + .setMaxVersion((short) 7); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, cluster, AdminClientConfig.RETRY_BACKOFF_MS_CONFIG, "0")) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(Arrays.asList(findCoordinatorV3, offsetFetchV7))); + env.kafkaClient().prepareResponse(prepareOldFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode())); + env.kafkaClient().prepareResponse(prepareOldFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + env.kafkaClient().prepareResponse(prepareOldFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); + + ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(groupSpecs); + + // Fail the first request in order to ensure that the group is not batched when retried. + sendOffsetFetchResponse(env.kafkaClient(), groupSpecs, false, Errors.COORDINATOR_LOAD_IN_PROGRESS); + + sendOffsetFetchResponse(env.kafkaClient(), groupSpecs, false, Errors.NONE); + sendOffsetFetchResponse(env.kafkaClient(), groupSpecs, false, Errors.NONE); + + verifyListOffsetsForMultipleGroups(groupSpecs, result); + } + } + + @Test + public void testBatchedListConsumerGroupOffsetsWithNoOffsetFetchBatching() throws Exception { + Cluster cluster = mockCluster(1, 0); + Time time = new MockTime(); + Map groupSpecs = batchedListConsumerGroupOffsetsSpec(); + + ApiVersion offsetFetchV7 = new ApiVersion() + .setApiKey(ApiKeys.OFFSET_FETCH.id) + .setMinVersion((short) 0) + .setMaxVersion((short) 7); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, cluster, AdminClientConfig.RETRY_BACKOFF_MS_CONFIG, "0")) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(Collections.singleton(offsetFetchV7))); + env.kafkaClient().prepareResponse(prepareBatchedFindCoordinatorResponse(Errors.NONE, env.cluster().controller(), groupSpecs.keySet())); + // Prepare a response to force client to attempt batched request creation that throws + // NoBatchedOffsetFetchRequestException. This triggers creation of non-batched requests. + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); + + ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(groupSpecs); + + // The request handler attempts both FindCoordinator and OffsetFetch requests. This seems + // ok since since we expect this scenario only during upgrades from versions < 3.0.0 where + // some upgraded brokers could handle batched FindCoordinator while non-upgraded coordinators + // rejected batched OffsetFetch requests. + sendFindCoordinatorResponse(env.kafkaClient(), env.cluster().controller()); + sendFindCoordinatorResponse(env.kafkaClient(), env.cluster().controller()); + sendOffsetFetchResponse(env.kafkaClient(), groupSpecs, false, Errors.NONE); + sendOffsetFetchResponse(env.kafkaClient(), groupSpecs, false, Errors.NONE); + + verifyListOffsetsForMultipleGroups(groupSpecs, result); + } + } + + private Map batchedListConsumerGroupOffsetsSpec() { + Set groupAPartitions = Collections.singleton(new TopicPartition("A", 1)); + Set groupBPartitions = Collections.singleton(new TopicPartition("B", 2)); + + ListConsumerGroupOffsetsSpec groupASpec = new ListConsumerGroupOffsetsSpec().topicPartitions(groupAPartitions); + ListConsumerGroupOffsetsSpec groupBSpec = new ListConsumerGroupOffsetsSpec().topicPartitions(groupBPartitions); + return Utils.mkMap(Utils.mkEntry("groupA", groupASpec), Utils.mkEntry("groupB", groupBSpec)); + } + + private void waitForRequest(MockClient mockClient, ApiKeys apiKeys) throws Exception { + TestUtils.waitForCondition(() -> { + ClientRequest clientRequest = mockClient.requests().peek(); + return clientRequest != null && clientRequest.apiKey() == apiKeys; + }, "Failed awaiting " + apiKeys + " request"); + } + + private void sendFindCoordinatorResponse(MockClient mockClient, Node coordinator) throws Exception { + waitForRequest(mockClient, ApiKeys.FIND_COORDINATOR); + + ClientRequest clientRequest = mockClient.requests().peek(); + FindCoordinatorRequestData data = ((FindCoordinatorRequest.Builder) clientRequest.requestBuilder()).data(); + mockClient.respond(prepareFindCoordinatorResponse(Errors.NONE, data.key(), coordinator)); + } + + private void sendOffsetFetchResponse(MockClient mockClient, Map groupSpecs, boolean batched, Errors error) throws Exception { + waitForRequest(mockClient, ApiKeys.OFFSET_FETCH); + + ClientRequest clientRequest = mockClient.requests().peek(); + OffsetFetchRequestData data = ((OffsetFetchRequest.Builder) clientRequest.requestBuilder()).data; + Map> results = new HashMap<>(); + Map errors = new HashMap<>(); + data.groups().forEach(group -> { + Map partitionResults = new HashMap<>(); + for (TopicPartition tp : groupSpecs.get(group.groupId()).topicPartitions()) { + partitionResults.put(tp, new PartitionData(10, Optional.empty(), "", Errors.NONE)); + } + results.put(group.groupId(), partitionResults); + errors.put(group.groupId(), error); + }); + if (!batched) { + assertEquals(1, data.groups().size()); + mockClient.respond(new OffsetFetchResponse(THROTTLE, error, results.values().iterator().next())); + } else + mockClient.respond(new OffsetFetchResponse(THROTTLE, errors, results)); + } + + private void verifyListOffsetsForMultipleGroups(Map groupSpecs, + ListConsumerGroupOffsetsResult result) throws Exception { + assertEquals(groupSpecs.size(), result.all().get(10, TimeUnit.SECONDS).size()); + for (Map.Entry entry : groupSpecs.entrySet()) { + assertEquals(entry.getValue().topicPartitions(), + result.partitionsToOffsetAndMetadata(entry.getKey()).get().keySet()); + } + } + @Test public void testDeleteConsumerGroupsNumRetries() throws Exception { final Cluster cluster = mockCluster(3, 0); @@ -4117,6 +4295,13 @@ public void testRemoveMembersFromGroupReason() throws Exception { testRemoveMembersFromGroup("testing remove members reason", "testing remove members reason"); } + @Test + public void testRemoveMembersFromGroupTruncatesReason() throws Exception { + final String reason = "Very looooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooong reason that is 271 characters long to make sure that length limit logic handles the scenario nicely"; + final String truncatedReason = reason.substring(0, 255); + testRemoveMembersFromGroup(reason, truncatedReason); + } + @Test public void testRemoveMembersFromGroupDefaultReason() throws Exception { testRemoveMembersFromGroup(null, DEFAULT_LEAVE_GROUP_REASON); @@ -5007,6 +5192,29 @@ public void testDescribeMetadataQuorumSuccess() throws Exception { } } + @Test + public void testDescribeMetadataQuorumRetriableError() throws Exception { + try (final AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(ApiKeys.DESCRIBE_QUORUM.id, + ApiKeys.DESCRIBE_QUORUM.oldestVersion(), + ApiKeys.DESCRIBE_QUORUM.latestVersion())); + + // First request fails with a NOT_LEADER_OR_FOLLOWER error (which is retriable) + env.kafkaClient().prepareResponse( + body -> body instanceof DescribeQuorumRequest, + prepareDescribeQuorumResponse(Errors.NONE, Errors.NOT_LEADER_OR_FOLLOWER, false, false, false, false, false)); + + // The second request succeeds + env.kafkaClient().prepareResponse( + body -> body instanceof DescribeQuorumRequest, + prepareDescribeQuorumResponse(Errors.NONE, Errors.NONE, false, false, false, false, false)); + + KafkaFuture future = env.adminClient().describeMetadataQuorum().quorumInfo(); + QuorumInfo quorumInfo = future.get(); + assertEquals(defaultQuorumInfo(false), quorumInfo); + } + } + @Test public void testDescribeMetadataQuorumFailure() { try (final AdminClientUnitTestEnv env = mockClientEnv()) { @@ -6537,6 +6745,12 @@ private DescribeLogDirsResponse prepareDescribeLogDirsResponse(Errors error, Str .setLogDir(logDir)))); } + private OffsetFetchResponse offsetFetchResponse(Errors error, Map responseData) { + return new OffsetFetchResponse(THROTTLE, + Collections.singletonMap(GROUP_ID, error), + Collections.singletonMap(GROUP_ID, responseData)); + } + private static MemberDescription convertToMemberDescriptions(DescribedGroupMember member, MemberAssignment assignment) { return new MemberDescription(member.memberId(), diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index ef858c5003d42..32b659fe4ae53 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.admin; import org.apache.kafka.clients.admin.DescribeReplicaLogDirsResult.ReplicaLogDirInfo; +import org.apache.kafka.clients.admin.internals.CoordinatorKey; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.ElectionType; import org.apache.kafka.common.KafkaException; @@ -37,6 +38,7 @@ import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.InvalidReplicationFactorException; import org.apache.kafka.common.errors.InvalidRequestException; +import org.apache.kafka.common.errors.InvalidUpdateVersionException; import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.errors.ReplicaNotAvailableException; import org.apache.kafka.common.errors.TimeoutException; @@ -76,6 +78,9 @@ public class MockAdminClient extends AdminClient { private final Map endOffsets; private final Map committedOffsets; private final boolean usingRaftController; + private final Map featureLevels; + private final Map minSupportedFeatureLevels; + private final Map maxSupportedFeatureLevels; private final String clusterId; private final List> brokerLogDirs; private final List> brokerConfigs; @@ -101,6 +106,9 @@ public static class Builder { private Short defaultPartitions; private boolean usingRaftController = false; private Integer defaultReplicationFactor; + private Map featureLevels = Collections.emptyMap(); + private Map minSupportedFeatureLevels = Collections.emptyMap(); + private Map maxSupportedFeatureLevels = Collections.emptyMap(); public Builder() { numBrokers(1); @@ -155,6 +163,21 @@ public Builder defaultPartitions(short numPartitions) { return this; } + public Builder featureLevels(Map featureLevels) { + this.featureLevels = featureLevels; + return this; + } + + public Builder minSupportedFeatureLevels(Map minSupportedFeatureLevels) { + this.minSupportedFeatureLevels = minSupportedFeatureLevels; + return this; + } + + public Builder maxSupportedFeatureLevels(Map maxSupportedFeatureLevels) { + this.maxSupportedFeatureLevels = maxSupportedFeatureLevels; + return this; + } + public MockAdminClient build() { return new MockAdminClient(brokers, controller == null ? brokers.get(0) : controller, @@ -162,7 +185,10 @@ public MockAdminClient build() { defaultPartitions != null ? defaultPartitions.shortValue() : 1, defaultReplicationFactor != null ? defaultReplicationFactor.shortValue() : Math.min(brokers.size(), 3), brokerLogDirs, - usingRaftController); + usingRaftController, + featureLevels, + minSupportedFeatureLevels, + maxSupportedFeatureLevels); } } @@ -171,17 +197,30 @@ public MockAdminClient() { } public MockAdminClient(List brokers, Node controller) { - this(brokers, controller, DEFAULT_CLUSTER_ID, 1, brokers.size(), - Collections.nCopies(brokers.size(), DEFAULT_LOG_DIRS), false); - } - - private MockAdminClient(List brokers, - Node controller, - String clusterId, - int defaultPartitions, - int defaultReplicationFactor, - List> brokerLogDirs, - boolean usingRaftController) { + this(brokers, + controller, + DEFAULT_CLUSTER_ID, + 1, + brokers.size(), + Collections.nCopies(brokers.size(), DEFAULT_LOG_DIRS), + false, + Collections.emptyMap(), + Collections.emptyMap(), + Collections.emptyMap()); + } + + private MockAdminClient( + List brokers, + Node controller, + String clusterId, + int defaultPartitions, + int defaultReplicationFactor, + List> brokerLogDirs, + boolean usingRaftController, + Map featureLevels, + Map minSupportedFeatureLevels, + Map maxSupportedFeatureLevels + ) { this.brokers = brokers; controller(controller); this.clusterId = clusterId; @@ -198,6 +237,9 @@ private MockAdminClient(List brokers, this.endOffsets = new HashMap<>(); this.committedOffsets = new HashMap<>(); this.usingRaftController = usingRaftController; + this.featureLevels = new HashMap<>(featureLevels); + this.minSupportedFeatureLevels = new HashMap<>(minSupportedFeatureLevels); + this.maxSupportedFeatureLevels = new HashMap<>(maxSupportedFeatureLevels); } synchronized public void controller(Node controller) { @@ -583,24 +625,29 @@ synchronized public ListConsumerGroupsResult listConsumerGroups(ListConsumerGrou } @Override - synchronized public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, ListConsumerGroupOffsetsOptions options) { - // ignoring the groupId and assume one test would only work on one group only + synchronized public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map groupSpecs, ListConsumerGroupOffsetsOptions options) { + // ignoring the groups and assume one test would only work on one group only + if (groupSpecs.size() != 1) + throw new UnsupportedOperationException("Not implemented yet"); + + String group = groupSpecs.keySet().iterator().next(); + Collection topicPartitions = groupSpecs.get(group).topicPartitions(); final KafkaFutureImpl> future = new KafkaFutureImpl<>(); if (listConsumerGroupOffsetsException != null) { future.completeExceptionally(listConsumerGroupOffsetsException); } else { - if (options.topicPartitions().isEmpty()) { + if (topicPartitions.isEmpty()) { future.complete(committedOffsets.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, entry -> new OffsetAndMetadata(entry.getValue())))); } else { future.complete(committedOffsets.entrySet().stream() - .filter(entry -> options.topicPartitions().contains(entry.getKey())) + .filter(entry -> topicPartitions.contains(entry.getKey())) .collect(Collectors.toMap(Map.Entry::getKey, entry -> new OffsetAndMetadata(entry.getValue())))); } } - return new ListConsumerGroupOffsetsResult(future); + return new ListConsumerGroupOffsetsResult(Collections.singletonMap(CoordinatorKey.byGroupId(group), future)); } @Override @@ -989,12 +1036,79 @@ public DescribeMetadataQuorumResult describeMetadataQuorum(DescribeMetadataQuoru @Override public DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options) { - throw new UnsupportedOperationException("Not implemented yet"); + Map finalizedFeatures = new HashMap<>(); + Map supportedFeatures = new HashMap<>(); + for (Map.Entry entry : featureLevels.entrySet()) { + finalizedFeatures.put(entry.getKey(), new FinalizedVersionRange( + entry.getValue(), entry.getValue())); + supportedFeatures.put(entry.getKey(), new SupportedVersionRange( + minSupportedFeatureLevels.get(entry.getKey()), + maxSupportedFeatureLevels.get(entry.getKey()))); + } + return new DescribeFeaturesResult(KafkaFuture.completedFuture( + new FeatureMetadata(finalizedFeatures, + Optional.of(123L), + supportedFeatures))); } @Override - public UpdateFeaturesResult updateFeatures(Map featureUpdates, UpdateFeaturesOptions options) { - throw new UnsupportedOperationException("Not implemented yet"); + public UpdateFeaturesResult updateFeatures( + Map featureUpdates, + UpdateFeaturesOptions options + ) { + Map> results = new HashMap<>(); + for (Map.Entry entry : featureUpdates.entrySet()) { + KafkaFutureImpl future = new KafkaFutureImpl(); + String feature = entry.getKey(); + try { + short cur = featureLevels.getOrDefault(feature, (short) 0); + short next = entry.getValue().maxVersionLevel(); + short min = minSupportedFeatureLevels.getOrDefault(feature, (short) 0); + short max = maxSupportedFeatureLevels.getOrDefault(feature, (short) 0); + switch (entry.getValue().upgradeType()) { + case UNKNOWN: + throw new InvalidRequestException("Invalid upgrade type."); + case UPGRADE: + if (cur > next) { + throw new InvalidUpdateVersionException("Can't upgrade to lower version."); + } + break; + case SAFE_DOWNGRADE: + if (cur < next) { + throw new InvalidUpdateVersionException("Can't downgrade to newer version."); + } + break; + case UNSAFE_DOWNGRADE: + if (cur < next) { + throw new InvalidUpdateVersionException("Can't downgrade to newer version."); + } + while (next != cur) { + // Simulate a scenario where all the even feature levels unsafe to downgrade from. + if (cur % 2 == 0) { + if (entry.getValue().upgradeType() == FeatureUpdate.UpgradeType.SAFE_DOWNGRADE) { + throw new InvalidUpdateVersionException("Unable to perform a safe downgrade."); + } + } + cur--; + } + break; + } + if (next < min) { + throw new InvalidUpdateVersionException("Can't downgrade below " + min); + } + if (next > max) { + throw new InvalidUpdateVersionException("Can't upgrade above " + max); + } + if (!options.validateOnly()) { + featureLevels.put(feature, next); + } + future.complete(null); + } catch (Exception e) { + future.completeExceptionally(e); + } + results.put(feature, future); + } + return new UpdateFeaturesResult(results); } @Override 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 27597ce035b00..95fabb3fc2a2f 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 @@ -24,52 +24,140 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsSpec; +import org.apache.kafka.clients.admin.internals.AdminApiHandler.RequestAndKeys; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.GroupIdNotFoundException; import org.apache.kafka.common.errors.InvalidGroupIdException; +import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestGroup; +import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestTopics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.OffsetFetchRequest; import org.apache.kafka.common.requests.OffsetFetchResponse; import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Utils; import org.junit.jupiter.api.Test; public class ListConsumerGroupOffsetsHandlerTest { private final LogContext logContext = new LogContext(); - private final String groupId = "group-id"; + private final int throttleMs = 10; + private final String groupZero = "group0"; + private final String groupOne = "group1"; + private final String groupTwo = "group2"; + private final List groups = Arrays.asList(groupZero, groupOne, groupTwo); private final TopicPartition t0p0 = new TopicPartition("t0", 0); private final TopicPartition t0p1 = new TopicPartition("t0", 1); private final TopicPartition t1p0 = new TopicPartition("t1", 0); private final TopicPartition t1p1 = new TopicPartition("t1", 1); - private final List tps = Arrays.asList(t0p0, t0p1, t1p0, t1p1); + private final TopicPartition t2p0 = new TopicPartition("t2", 0); + private final TopicPartition t2p1 = new TopicPartition("t2", 1); + private final TopicPartition t2p2 = new TopicPartition("t2", 2); + private final Map singleRequestMap = Collections.singletonMap(groupZero, + new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t0p1, t1p0, t1p1))); + private final Map batchedRequestMap = + new HashMap() {{ + put(groupZero, new ListConsumerGroupOffsetsSpec().topicPartitions(singletonList(t0p0))); + put(groupOne, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t1p0, t1p1))); + put(groupTwo, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t1p0, t1p1, t2p0, t2p1, t2p2))); + }}; @Test public void testBuildRequest() { - ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, tps, logContext); - OffsetFetchRequest request = handler.buildBatchedRequest(1, singleton(CoordinatorKey.byGroupId(groupId))).build(); - assertEquals(groupId, request.data().groups().get(0).groupId()); + ListConsumerGroupOffsetsHandler handler = + new ListConsumerGroupOffsetsHandler(singleRequestMap, false, logContext); + OffsetFetchRequest request = handler.buildBatchedRequest(coordinatorKeys(groupZero)).build(); + assertEquals(groupZero, request.data().groups().get(0).groupId()); assertEquals(2, request.data().groups().get(0).topics().size()); assertEquals(2, request.data().groups().get(0).topics().get(0).partitionIndexes().size()); assertEquals(2, request.data().groups().get(0).topics().get(1).partitionIndexes().size()); } + @Test + public void testBuildRequestWithMultipleGroups() { + Map requestMap = new HashMap<>(this.batchedRequestMap); + String groupThree = "group3"; + requestMap.put(groupThree, new ListConsumerGroupOffsetsSpec() + .topicPartitions(Arrays.asList(new TopicPartition("t3", 0), new TopicPartition("t3", 1)))); + + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(requestMap, false, logContext); + OffsetFetchRequest request1 = handler.buildBatchedRequest(coordinatorKeys(groupZero, groupOne, groupTwo)).build(); + assertEquals(Utils.mkSet(groupZero, groupOne, groupTwo), requestGroups(request1)); + + OffsetFetchRequest request2 = handler.buildBatchedRequest(coordinatorKeys(groupThree)).build(); + assertEquals(Utils.mkSet(groupThree), requestGroups(request2)); + + Map builtRequests = new HashMap<>(); + request1.groupIdsToPartitions().forEach((group, partitions) -> + builtRequests.put(group, new ListConsumerGroupOffsetsSpec().topicPartitions(partitions))); + request2.groupIdsToPartitions().forEach((group, partitions) -> + builtRequests.put(group, new ListConsumerGroupOffsetsSpec().topicPartitions(partitions))); + + assertEquals(requestMap, builtRequests); + Map> groupIdsToTopics = request1.groupIdsToTopics(); + + assertEquals(3, groupIdsToTopics.size()); + assertEquals(1, groupIdsToTopics.get(groupZero).size()); + assertEquals(2, groupIdsToTopics.get(groupOne).size()); + assertEquals(3, groupIdsToTopics.get(groupTwo).size()); + + assertEquals(1, groupIdsToTopics.get(groupZero).get(0).partitionIndexes().size()); + assertEquals(1, groupIdsToTopics.get(groupOne).get(0).partitionIndexes().size()); + assertEquals(2, groupIdsToTopics.get(groupOne).get(1).partitionIndexes().size()); + assertEquals(1, groupIdsToTopics.get(groupTwo).get(0).partitionIndexes().size()); + assertEquals(2, groupIdsToTopics.get(groupTwo).get(1).partitionIndexes().size()); + assertEquals(3, groupIdsToTopics.get(groupTwo).get(2).partitionIndexes().size()); + + groupIdsToTopics = request2.groupIdsToTopics(); + assertEquals(1, groupIdsToTopics.size()); + assertEquals(1, groupIdsToTopics.get(groupThree).size()); + assertEquals(2, groupIdsToTopics.get(groupThree).get(0).partitionIndexes().size()); + } + + @Test + public void testBuildRequestBatchGroups() { + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(batchedRequestMap, false, logContext); + Collection> requests = handler.buildRequest(1, coordinatorKeys(groupZero, groupOne, groupTwo)); + assertEquals(1, requests.size()); + assertEquals(Utils.mkSet(groupZero, groupOne, groupTwo), requestGroups((OffsetFetchRequest) requests.iterator().next().request.build())); + } + + @Test + public void testBuildRequestDoesNotBatchGroup() { + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(batchedRequestMap, false, logContext); + // Disable batching. + ((CoordinatorStrategy) handler.lookupStrategy()).disableBatch(); + Collection> requests = handler.buildRequest(1, coordinatorKeys(groupZero, groupOne, groupTwo)); + assertEquals(3, requests.size()); + assertEquals( + Utils.mkSet(Utils.mkSet(groupZero), Utils.mkSet(groupOne), Utils.mkSet(groupTwo)), + requests.stream().map(requestAndKey -> requestGroups((OffsetFetchRequest) requestAndKey.request.build())).collect(Collectors.toSet()) + ); + } + @Test public void testSuccessfulHandleResponse() { Map expected = new HashMap<>(); assertCompleted(handleWithError(Errors.NONE), expected); } - @Test public void testSuccessfulHandleResponseWithOnePartitionError() { Map expectedResult = Collections.singletonMap(t0p0, new OffsetAndMetadata(10L)); @@ -80,17 +168,62 @@ public void testSuccessfulHandleResponseWithOnePartitionError() { assertCompleted(handleWithPartitionError(Errors.UNSTABLE_OFFSET_COMMIT), expectedResult); } + @Test + public void testSuccessfulHandleResponseWithOnePartitionErrorWithMultipleGroups() { + Map offsetAndMetadataMapZero = + Collections.singletonMap(t0p0, new OffsetAndMetadata(10L)); + Map offsetAndMetadataMapOne = + Collections.singletonMap(t1p1, new OffsetAndMetadata(10L)); + Map offsetAndMetadataMapTwo = + Collections.singletonMap(t2p2, new OffsetAndMetadata(10L)); + Map> expectedResult = + new HashMap>() {{ + put(groupZero, offsetAndMetadataMapZero); + put(groupOne, offsetAndMetadataMapOne); + put(groupTwo, offsetAndMetadataMapTwo); + }}; + + assertCompletedForMultipleGroups( + handleWithPartitionErrorMultipleGroups(Errors.UNKNOWN_TOPIC_OR_PARTITION), expectedResult); + assertCompletedForMultipleGroups( + handleWithPartitionErrorMultipleGroups(Errors.TOPIC_AUTHORIZATION_FAILED), expectedResult); + assertCompletedForMultipleGroups( + handleWithPartitionErrorMultipleGroups(Errors.UNSTABLE_OFFSET_COMMIT), expectedResult); + } + + @Test + public void testSuccessfulHandleResponseWithMultipleGroups() { + Map> expected = new HashMap<>(); + Map errorMap = errorMap(groups, Errors.NONE); + assertCompletedForMultipleGroups(handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap), expected); + } + @Test public void testUnmappedHandleResponse() { assertUnmapped(handleWithError(Errors.COORDINATOR_NOT_AVAILABLE)); assertUnmapped(handleWithError(Errors.NOT_COORDINATOR)); } + @Test + public void testUnmappedHandleResponseWithMultipleGroups() { + Map errorMap = new HashMap<>(); + errorMap.put(groupZero, Errors.NOT_COORDINATOR); + errorMap.put(groupOne, Errors.COORDINATOR_NOT_AVAILABLE); + errorMap.put(groupTwo, Errors.NOT_COORDINATOR); + assertUnmappedWithMultipleGroups(handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap)); + } + @Test public void testRetriableHandleResponse() { assertRetriable(handleWithError(Errors.COORDINATOR_LOAD_IN_PROGRESS)); } + @Test + public void testRetriableHandleResponseWithMultipleGroups() { + Map errorMap = errorMap(groups, Errors.COORDINATOR_LOAD_IN_PROGRESS); + assertRetriable(handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap)); + } + @Test public void testFailedHandleResponse() { assertFailed(GroupAuthorizationException.class, handleWithError(Errors.GROUP_AUTHORIZATION_FAILED)); @@ -98,10 +231,50 @@ public void testFailedHandleResponse() { assertFailed(InvalidGroupIdException.class, handleWithError(Errors.INVALID_GROUP_ID)); } + @Test + public void testFailedHandleResponseWithMultipleGroups() { + Map errorMap = new HashMap<>(); + errorMap.put(groupZero, Errors.GROUP_AUTHORIZATION_FAILED); + errorMap.put(groupOne, Errors.GROUP_ID_NOT_FOUND); + errorMap.put(groupTwo, Errors.INVALID_GROUP_ID); + Map> groupToExceptionMap = new HashMap<>(); + groupToExceptionMap.put(groupZero, GroupAuthorizationException.class); + groupToExceptionMap.put(groupOne, GroupIdNotFoundException.class); + groupToExceptionMap.put(groupTwo, InvalidGroupIdException.class); + assertFailedForMultipleGroups(groupToExceptionMap, + handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap)); + } + private OffsetFetchResponse buildResponse(Errors error) { - Map responseData = new HashMap<>(); - OffsetFetchResponse response = new OffsetFetchResponse(error, responseData); - return response; + return new OffsetFetchResponse( + throttleMs, + Collections.singletonMap(groupZero, error), + Collections.singletonMap(groupZero, new HashMap<>())); + } + + private OffsetFetchResponse buildResponseWithMultipleGroups( + Map errorMap, + Map> responseData + ) { + return new OffsetFetchResponse(throttleMs, errorMap, responseData); + } + + private AdminApiHandler.ApiResult> handleWithErrorWithMultipleGroups( + Map errorMap, + Map groupSpecs + ) { + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupSpecs, false, logContext); + Map> responseData = new HashMap<>(); + for (String group : errorMap.keySet()) { + responseData.put(group, new HashMap<>()); + } + OffsetFetchResponse response = buildResponseWithMultipleGroups(errorMap, responseData); + return handler.handleResponse(new Node(1, "host", 1234), + errorMap.keySet() + .stream() + .map(CoordinatorKey::byGroupId) + .collect(Collectors.toSet()), + response); } private OffsetFetchResponse buildResponseWithPartitionError(Errors error) { @@ -110,24 +283,68 @@ private OffsetFetchResponse buildResponseWithPartitionError(Errors error) { responseData.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE)); responseData.put(t0p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); - OffsetFetchResponse response = new OffsetFetchResponse(Errors.NONE, responseData); - return response; + return new OffsetFetchResponse(Errors.NONE, responseData); + } + + private OffsetFetchResponse buildResponseWithPartitionErrorWithMultipleGroups(Errors error) { + Map responseDataZero = new HashMap<>(); + responseDataZero.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE)); + + Map responseDataOne = new HashMap<>(); + responseDataOne.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataOne.put(t1p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataOne.put(t1p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE)); + + Map responseDataTwo = new HashMap<>(); + responseDataTwo.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataTwo.put(t1p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataTwo.put(t1p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataTwo.put(t2p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataTwo.put(t2p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); + responseDataTwo.put(t2p2, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE)); + + Map> responseData = + new HashMap>() {{ + put(groupZero, responseDataZero); + put(groupOne, responseDataOne); + put(groupTwo, responseDataTwo); + }}; + + Map errorMap = errorMap(groups, Errors.NONE); + return new OffsetFetchResponse(0, errorMap, responseData); } private AdminApiHandler.ApiResult> handleWithPartitionError( Errors error ) { - ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, tps, logContext); + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(singleRequestMap, + false, logContext); OffsetFetchResponse response = buildResponseWithPartitionError(error); - return handler.handleResponse(new Node(1, "host", 1234), singleton(CoordinatorKey.byGroupId(groupId)), response); + return handler.handleResponse(new Node(1, "host", 1234), + singleton(CoordinatorKey.byGroupId(groupZero)), response); + } + + private AdminApiHandler.ApiResult> handleWithPartitionErrorMultipleGroups( + Errors error + ) { + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler( + batchedRequestMap, false, logContext); + OffsetFetchResponse response = buildResponseWithPartitionErrorWithMultipleGroups(error); + return handler.handleResponse( + new Node(1, "host", 1234), + coordinatorKeys(groupZero, groupOne, groupTwo), + response); } private AdminApiHandler.ApiResult> handleWithError( Errors error ) { - ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupId, tps, logContext); + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler( + singleRequestMap, false, logContext); OffsetFetchResponse response = buildResponse(error); - return handler.handleResponse(new Node(1, "host", 1234), singleton(CoordinatorKey.byGroupId(groupId)), response); + return handler.handleResponse(new Node(1, "host", 1234), + singleton(CoordinatorKey.byGroupId(groupZero)), + response); } private void assertUnmapped( @@ -135,11 +352,19 @@ private void assertUnmapped( ) { assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptySet(), result.failedKeys.keySet()); - assertEquals(singletonList(CoordinatorKey.byGroupId(groupId)), result.unmappedKeys); + assertEquals(singletonList(CoordinatorKey.byGroupId(groupZero)), result.unmappedKeys); + } + + private void assertUnmappedWithMultipleGroups( + AdminApiHandler.ApiResult> result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(coordinatorKeys(groupZero, groupOne, groupTwo), new HashSet<>(result.unmappedKeys)); } private void assertRetriable( - AdminApiHandler.ApiResult> result + AdminApiHandler.ApiResult> result ) { assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptySet(), result.failedKeys.keySet()); @@ -150,21 +375,64 @@ private void assertCompleted( AdminApiHandler.ApiResult> result, Map expected ) { - CoordinatorKey key = CoordinatorKey.byGroupId(groupId); + CoordinatorKey key = CoordinatorKey.byGroupId(groupZero); assertEquals(emptySet(), result.failedKeys.keySet()); assertEquals(emptyList(), result.unmappedKeys); assertEquals(singleton(key), result.completedKeys.keySet()); - assertEquals(expected, result.completedKeys.get(CoordinatorKey.byGroupId(groupId))); + assertEquals(expected, result.completedKeys.get(key)); + } + + private void assertCompletedForMultipleGroups( + AdminApiHandler.ApiResult> result, + Map> expected + ) { + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + for (String g : expected.keySet()) { + CoordinatorKey key = CoordinatorKey.byGroupId(g); + assertTrue(result.completedKeys.containsKey(key)); + assertEquals(expected.get(g), result.completedKeys.get(key)); + } } private void assertFailed( Class expectedExceptionType, AdminApiHandler.ApiResult> result ) { - CoordinatorKey key = CoordinatorKey.byGroupId(groupId); + CoordinatorKey key = CoordinatorKey.byGroupId(groupZero); assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptyList(), result.unmappedKeys); assertEquals(singleton(key), result.failedKeys.keySet()); assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); } + + private void assertFailedForMultipleGroups( + Map> groupToExceptionMap, + AdminApiHandler.ApiResult> result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + for (String g : groupToExceptionMap.keySet()) { + CoordinatorKey key = CoordinatorKey.byGroupId(g); + assertTrue(result.failedKeys.containsKey(key)); + assertTrue(groupToExceptionMap.get(g).isInstance(result.failedKeys.get(key))); + } + } + + private Set coordinatorKeys(String... groups) { + return Stream.of(groups) + .map(CoordinatorKey::byGroupId) + .collect(Collectors.toSet()); + } + + private Set requestGroups(OffsetFetchRequest request) { + return request.data().groups() + .stream() + .map(OffsetFetchRequestGroup::groupId) + .collect(Collectors.toSet()); + } + + private Map errorMap(Collection groups, Errors error) { + return groups.stream().collect(Collectors.toMap(Function.identity(), unused -> error)); + } } 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 da3acf4983471..e7f25345c6da0 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 @@ -172,6 +172,7 @@ public class KafkaConsumerTest { // Set auto commit interval lower than heartbeat so we don't need to deal with // a concurrent heartbeat request private final int autoCommitIntervalMs = 500; + private final int throttleMs = 10; private final String groupId = "mock-group"; private final String memberId = "memberId"; @@ -2434,7 +2435,10 @@ private OffsetFetchResponse offsetResponse(Map offsets, Er partitionData.put(entry.getKey(), new OffsetFetchResponse.PartitionData(entry.getValue(), Optional.empty(), "", error)); } - return new OffsetFetchResponse(Errors.NONE, partitionData); + return new OffsetFetchResponse( + throttleMs, + Collections.singletonMap(groupId, Errors.NONE), + Collections.singletonMap(groupId, partitionData)); } private ListOffsetsResponse listOffsetsResponse(Map offsets) { 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 6812af29ce5de..0745b99749f51 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 @@ -67,7 +67,6 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -272,6 +271,21 @@ public void testCoordinatorDiscoveryBackoff() { assertTrue(endTime - initialTime >= RETRY_BACKOFF_MS); } + @Test + public void testWakeupFromEnsureCoordinatorReady() { + setupCoordinator(); + + consumerClient.wakeup(); + + // No wakeup should occur from the async variation. + coordinator.ensureCoordinatorReadyAsync(); + + // But should wakeup in sync variation even if timer is 0. + assertThrows(WakeupException.class, () -> { + coordinator.ensureCoordinatorReady(mockTime.timer(0)); + }); + } + @Test public void testTimeoutAndRetryJoinGroupIfNeeded() throws Exception { setupCoordinator(); @@ -489,54 +503,6 @@ public void testRetainMemberIdAfterSyncGroupDisconnect() { ensureActiveGroup(rejoinedGeneration, memberId); } - @Test - public void testResetGenerationIdAfterSyncGroupFailedWithRebalanceInProgress() throws InterruptedException, ExecutionException { - setupCoordinator(); - - String memberId = "memberId"; - int generation = 5; - - // Rebalance once to initialize the generation and memberId - mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); - expectJoinGroup("", generation, memberId); - expectSyncGroup(generation, memberId); - ensureActiveGroup(generation, memberId); - - // Force a rebalance - coordinator.requestRejoin("Manual test trigger"); - assertTrue(coordinator.rejoinNeededOrPending()); - - ExecutorService executor = Executors.newFixedThreadPool(1); - try { - // Return RebalanceInProgress in syncGroup - int rejoinedGeneration = 10; - expectJoinGroup(memberId, rejoinedGeneration, memberId); - expectRebalanceInProgressForSyncGroup(rejoinedGeneration, memberId); - Future secondJoin = executor.submit(() -> - coordinator.ensureActiveGroup(mockTime.timer(Integer.MAX_VALUE))); - - TestUtils.waitForCondition(() -> { - AbstractCoordinator.Generation currentGeneration = coordinator.generation(); - return currentGeneration.generationId == AbstractCoordinator.Generation.NO_GENERATION.generationId && - currentGeneration.memberId.equals(memberId); - }, 2000, "Generation should be reset"); - - rejoinedGeneration = 20; - expectSyncGroup(rejoinedGeneration, memberId); - mockClient.respond(joinGroupFollowerResponse( - rejoinedGeneration, - memberId, - "leaderId", - Errors.NONE, - PROTOCOL_TYPE - )); - assertTrue(secondJoin.get()); - } finally { - executor.shutdownNow(); - executor.awaitTermination(1000, TimeUnit.MILLISECONDS); - } - } - @Test public void testRejoinReason() { setupCoordinator(); @@ -571,6 +537,15 @@ public void testRejoinReason() { expectSyncGroup(generation, memberId); ensureActiveGroup(generation, memberId); assertEquals("", coordinator.rejoinReason()); + + // check limit length of reason field + final String reason = "Very looooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooong reason that is 271 characters long to make sure that length limit logic handles the scenario nicely"; + final String truncatedReason = reason.substring(0, 255); + expectJoinGroup(memberId, truncatedReason, generation, memberId); + expectSyncGroup(generation, memberId); + coordinator.requestRejoin(reason); + ensureActiveGroup(generation, memberId); + assertEquals("", coordinator.rejoinReason()); } private void ensureActiveGroup( @@ -615,22 +590,6 @@ private void expectDisconnectInSyncGroup( }, null, true); } - private void expectRebalanceInProgressForSyncGroup( - int expectedGeneration, - String expectedMemberId - ) { - mockClient.prepareResponse(body -> { - if (!(body instanceof SyncGroupRequest)) { - return false; - } - SyncGroupRequestData syncGroupRequest = ((SyncGroupRequest) body).data(); - return syncGroupRequest.generationId() == expectedGeneration - && syncGroupRequest.memberId().equals(expectedMemberId) - && syncGroupRequest.protocolType().equals(PROTOCOL_TYPE) - && syncGroupRequest.protocolName().equals(PROTOCOL_NAME); - }, syncGroupResponse(Errors.REBALANCE_IN_PROGRESS, PROTOCOL_TYPE, PROTOCOL_NAME)); - } - private void expectDisconnectInJoinGroup( String expectedMemberId ) { @@ -1159,6 +1118,19 @@ public void testHandleNormalLeaveGroupResponse() { assertTrue(leaveGroupFuture.succeeded()); } + @Test + public void testHandleNormalLeaveGroupResponseAndTruncatedLeaveReason() { + MemberResponse memberResponse = new MemberResponse() + .setMemberId(memberId) + .setErrorCode(Errors.NONE.code()); + LeaveGroupResponse response = + leaveGroupResponse(Collections.singletonList(memberResponse)); + String leaveReason = "Very looooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooooong leaveReason that is 271 characters long to make sure that length limit logic handles the scenario nicely"; + RequestFuture leaveGroupFuture = setupLeaveGroup(response, leaveReason, leaveReason.substring(0, 255)); + assertNotNull(leaveGroupFuture); + assertTrue(leaveGroupFuture.succeeded()); + } + @Test public void testHandleMultipleMembersLeaveGroupResponse() { MemberResponse memberResponse = new MemberResponse() @@ -1193,6 +1165,12 @@ public void testHandleLeaveGroupResponseWithException() { } private RequestFuture setupLeaveGroup(LeaveGroupResponse leaveGroupResponse) { + return setupLeaveGroup(leaveGroupResponse, "test maybe leave group", "test maybe leave group"); + } + + private RequestFuture setupLeaveGroup(LeaveGroupResponse leaveGroupResponse, + String leaveReason, + String expectedLeaveReason) { setupCoordinator(RETRY_BACKOFF_MS, Integer.MAX_VALUE, Optional.empty()); mockClient.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); @@ -1204,11 +1182,11 @@ private RequestFuture setupLeaveGroup(LeaveGroupResponse leaveGroupRespons } LeaveGroupRequestData leaveGroupRequest = ((LeaveGroupRequest) body).data(); return leaveGroupRequest.members().get(0).memberId().equals(memberId) && - leaveGroupRequest.members().get(0).reason().equals("test maybe leave group"); + leaveGroupRequest.members().get(0).reason().equals(expectedLeaveReason); }, leaveGroupResponse); coordinator.ensureActiveGroup(); - return coordinator.maybeLeaveGroup("test maybe leave group"); + return coordinator.maybeLeaveGroup(leaveReason); } @Test @@ -1698,7 +1676,7 @@ protected Map onLeaderElected(String leaderId, } @Override - protected boolean onJoinPrepare(int generation, String memberId) { + protected boolean onJoinPrepare(Timer timer, int generation, String memberId) { onJoinPrepareInvokes++; return true; } 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 c65d33176fcd5..5e080b7721363 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 @@ -71,12 +71,14 @@ import org.apache.kafka.common.requests.OffsetCommitRequest; import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.requests.SyncGroupRequest; import org.apache.kafka.common.requests.SyncGroupResponse; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -140,6 +142,7 @@ public abstract class ConsumerCoordinatorTest { private final long retryBackoffMs = 100; private final int autoCommitIntervalMs = 2000; private final int requestTimeoutMs = 30000; + private final int throttleMs = 10; private final MockTime time = new MockTime(); private GroupRebalanceConfig rebalanceConfig; @@ -1299,9 +1302,71 @@ public void testForceMetadataDeleteForPatternSubscriptionDuringRebalance() { } } + @Test + public void testOnJoinPrepareWithOffsetCommitShouldSuccessAfterRetry() { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty(), false)) { + int generationId = 42; + String memberId = "consumer-42"; + + Timer pollTimer = time.timer(100L); + client.prepareResponse(offsetCommitResponse(singletonMap(t1p, Errors.UNKNOWN_TOPIC_OR_PARTITION))); + boolean res = coordinator.onJoinPrepare(pollTimer, generationId, memberId); + assertFalse(res); + + pollTimer = time.timer(100L); + client.prepareResponse(offsetCommitResponse(singletonMap(t1p, Errors.NONE))); + res = coordinator.onJoinPrepare(pollTimer, generationId, memberId); + assertTrue(res); + + assertFalse(client.hasPendingResponses()); + assertFalse(client.hasInFlightRequests()); + assertFalse(coordinator.coordinatorUnknown()); + } + } + + @Test + public void testOnJoinPrepareWithOffsetCommitShouldKeepJoinAfterNonRetryableException() { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty(), false)) { + int generationId = 42; + String memberId = "consumer-42"; + + Timer pollTimer = time.timer(100L); + client.prepareResponse(offsetCommitResponse(singletonMap(t1p, Errors.UNKNOWN_MEMBER_ID))); + boolean res = coordinator.onJoinPrepare(pollTimer, generationId, memberId); + assertTrue(res); + + assertFalse(client.hasPendingResponses()); + assertFalse(client.hasInFlightRequests()); + assertFalse(coordinator.coordinatorUnknown()); + } + } + + @Test + public void testOnJoinPrepareWithOffsetCommitShouldKeepJoinAfterRebalanceTimeout() { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty(), false)) { + int generationId = 42; + String memberId = "consumer-42"; + + Timer pollTimer = time.timer(100L); + time.sleep(150); + boolean res = coordinator.onJoinPrepare(pollTimer, generationId, memberId); + assertFalse(res); + + pollTimer = time.timer(100L); + time.sleep(rebalanceTimeoutMs); + client.respond(offsetCommitResponse(singletonMap(t1p, Errors.UNKNOWN_TOPIC_OR_PARTITION))); + res = coordinator.onJoinPrepare(pollTimer, generationId, memberId); + assertTrue(res); + + assertFalse(client.hasPendingResponses()); + assertFalse(client.hasInFlightRequests()); + assertFalse(coordinator.coordinatorUnknown()); + } + } + @Test public void testJoinPrepareWithDisableAutoCommit() { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"), true)) { coordinator.ensureActiveGroup(); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.NONE); @@ -1309,7 +1374,7 @@ public void testJoinPrepareWithDisableAutoCommit() { int generationId = 42; String memberId = "consumer-42"; - boolean res = coordinator.onJoinPrepare(generationId, memberId); + boolean res = coordinator.onJoinPrepare(time.timer(0L), generationId, memberId); assertTrue(res); assertTrue(client.hasPendingResponses()); @@ -1320,14 +1385,14 @@ public void testJoinPrepareWithDisableAutoCommit() { @Test public void testJoinPrepareAndCommitCompleted() { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.of("group-id"), true)) { coordinator.ensureActiveGroup(); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.NONE); int generationId = 42; String memberId = "consumer-42"; - boolean res = coordinator.onJoinPrepare(generationId, memberId); + boolean res = coordinator.onJoinPrepare(time.timer(0L), generationId, memberId); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(res); @@ -1339,7 +1404,7 @@ public void testJoinPrepareAndCommitCompleted() { @Test public void testJoinPrepareAndCommitWithCoordinatorNotAvailable() { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.of("group-id"), true)) { coordinator.ensureActiveGroup(); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.COORDINATOR_NOT_AVAILABLE); @@ -1347,7 +1412,7 @@ public void testJoinPrepareAndCommitWithCoordinatorNotAvailable() { int generationId = 42; String memberId = "consumer-42"; - boolean res = coordinator.onJoinPrepare(generationId, memberId); + boolean res = coordinator.onJoinPrepare(time.timer(0L), generationId, memberId); coordinator.invokeCompletedOffsetCommitCallbacks(); assertFalse(res); @@ -1359,7 +1424,7 @@ public void testJoinPrepareAndCommitWithCoordinatorNotAvailable() { @Test public void testJoinPrepareAndCommitWithUnknownMemberId() { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.of("group-id"), true)) { coordinator.ensureActiveGroup(); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.UNKNOWN_MEMBER_ID); @@ -1367,7 +1432,7 @@ public void testJoinPrepareAndCommitWithUnknownMemberId() { int generationId = 42; String memberId = "consumer-42"; - boolean res = coordinator.onJoinPrepare(generationId, memberId); + boolean res = coordinator.onJoinPrepare(time.timer(0L), generationId, memberId); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(res); @@ -2756,6 +2821,32 @@ public void testCommitOffsetFencedInstanceWithNewGeneration() { assertEquals(newGen, coordinator.generation()); } + @Test + public void testCommitOffsetShouldNotSetInstanceIdIfMemberIdIsUnknown() { + rebalanceConfig = buildRebalanceConfig(groupInstanceId); + ConsumerCoordinator coordinator = buildCoordinator( + rebalanceConfig, + new Metrics(), + assignors, + false, + subscriptions + ); + + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + coordinator.ensureCoordinatorReady(time.timer(5000)); + + client.prepareResponse(body -> { + OffsetCommitRequestData data = ((OffsetCommitRequest) body).data(); + return data.groupInstanceId() == null && data.memberId().isEmpty(); + }, offsetCommitResponse(Collections.emptyMap())); + + RequestFuture future = coordinator.sendOffsetCommitRequest(singletonMap(t1p, + new OffsetAndMetadata(100L, "metadata"))); + + assertTrue(consumerClient.poll(future, time.timer(5000))); + assertFalse(future.failed()); + } + @Test public void testCommitOffsetRebalanceInProgress() { // we cannot retry if a rebalance occurs before the commit completed @@ -2872,7 +2963,7 @@ public void testFetchCommittedOffsets() { OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, leaderEpoch, metadata, Errors.NONE); - client.prepareResponse(new OffsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); + client.prepareResponse(offsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); Map fetchedOffsets = coordinator.fetchCommittedOffsets(singleton(t1p), time.timer(Long.MAX_VALUE)); @@ -2888,7 +2979,7 @@ public void testTopicAuthorizationFailedInOffsetFetch() { OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(-1, Optional.empty(), "", Errors.TOPIC_AUTHORIZATION_FAILED); - client.prepareResponse(new OffsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); + client.prepareResponse(offsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); TopicAuthorizationException exception = assertThrows(TopicAuthorizationException.class, () -> coordinator.fetchCommittedOffsets(singleton(t1p), time.timer(Long.MAX_VALUE))); @@ -2901,7 +2992,7 @@ public void testRefreshOffsetLoadInProgress() { coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); subscriptions.assignFromUser(singleton(t1p)); - client.prepareResponse(offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS)); + client.prepareResponse(offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); @@ -2916,7 +3007,7 @@ public void testRefreshOffsetsGroupNotAuthorized() { coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); subscriptions.assignFromUser(singleton(t1p)); - client.prepareResponse(offsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED)); + client.prepareResponse(offsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED, Collections.emptyMap())); try { coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); fail("Expected group authorization error"); @@ -2959,7 +3050,7 @@ public void testRefreshOffsetNotCoordinatorForConsumer() { coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); subscriptions.assignFromUser(singleton(t1p)); - client.prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR)); + client.prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); @@ -3078,21 +3169,21 @@ public void run() { @Test public void testCloseDynamicAssignment() { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty(), true)) { gracefulCloseTest(coordinator, true); } } @Test public void testCloseManualAssignment() { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(false, true, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(false, true, Optional.empty(), true)) { gracefulCloseTest(coordinator, false); } } @Test public void testCloseCoordinatorNotKnownManualAssignment() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(false, true, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(false, true, Optional.empty(), true)) { makeCoordinatorUnknown(coordinator, Errors.NOT_COORDINATOR); time.sleep(autoCommitIntervalMs); closeVerifyTimeout(coordinator, 1000, 1000, 1000); @@ -3101,7 +3192,7 @@ public void testCloseCoordinatorNotKnownManualAssignment() throws Exception { @Test public void testCloseCoordinatorNotKnownNoCommits() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.empty(), true)) { makeCoordinatorUnknown(coordinator, Errors.NOT_COORDINATOR); closeVerifyTimeout(coordinator, 1000, 0, 0); } @@ -3109,7 +3200,7 @@ public void testCloseCoordinatorNotKnownNoCommits() throws Exception { @Test public void testCloseCoordinatorNotKnownWithCommits() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, Optional.empty(), true)) { makeCoordinatorUnknown(coordinator, Errors.NOT_COORDINATOR); time.sleep(autoCommitIntervalMs); closeVerifyTimeout(coordinator, 1000, 1000, 1000); @@ -3118,7 +3209,7 @@ public void testCloseCoordinatorNotKnownWithCommits() throws Exception { @Test public void testCloseCoordinatorUnavailableNoCommits() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.empty(), true)) { makeCoordinatorUnknown(coordinator, Errors.COORDINATOR_NOT_AVAILABLE); closeVerifyTimeout(coordinator, 1000, 0, 0); } @@ -3126,7 +3217,7 @@ public void testCloseCoordinatorUnavailableNoCommits() throws Exception { @Test public void testCloseTimeoutCoordinatorUnavailableForCommit() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId)) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId, true)) { makeCoordinatorUnknown(coordinator, Errors.COORDINATOR_NOT_AVAILABLE); time.sleep(autoCommitIntervalMs); closeVerifyTimeout(coordinator, 1000, 1000, 1000); @@ -3135,7 +3226,7 @@ public void testCloseTimeoutCoordinatorUnavailableForCommit() throws Exception { @Test public void testCloseMaxWaitCoordinatorUnavailableForCommit() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId)) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId, true)) { makeCoordinatorUnknown(coordinator, Errors.COORDINATOR_NOT_AVAILABLE); time.sleep(autoCommitIntervalMs); closeVerifyTimeout(coordinator, Long.MAX_VALUE, requestTimeoutMs, requestTimeoutMs); @@ -3144,7 +3235,7 @@ public void testCloseMaxWaitCoordinatorUnavailableForCommit() throws Exception { @Test public void testCloseNoResponseForCommit() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId)) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId, true)) { time.sleep(autoCommitIntervalMs); closeVerifyTimeout(coordinator, Long.MAX_VALUE, requestTimeoutMs, requestTimeoutMs); } @@ -3152,14 +3243,14 @@ public void testCloseNoResponseForCommit() throws Exception { @Test public void testCloseNoResponseForLeaveGroup() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.empty())) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.empty(), true)) { closeVerifyTimeout(coordinator, Long.MAX_VALUE, requestTimeoutMs, requestTimeoutMs); } } @Test public void testCloseNoWait() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId)) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId, true)) { time.sleep(autoCommitIntervalMs); closeVerifyTimeout(coordinator, 0, 0, 0); } @@ -3167,7 +3258,7 @@ public void testCloseNoWait() throws Exception { @Test public void testHeartbeatThreadClose() throws Exception { - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId)) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId, true)) { coordinator.ensureActiveGroup(); time.sleep(heartbeatIntervalMs + 100); Thread.yield(); // Give heartbeat thread a chance to attempt heartbeat @@ -3234,7 +3325,7 @@ public void testGetGroupMetadata() { assertEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, groupMetadata.memberId()); assertFalse(groupMetadata.groupInstanceId().isPresent()); - try (final ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId)) { + try (final ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, true, groupInstanceId, true)) { coordinator.ensureActiveGroup(); final ConsumerGroupMetadata joinedGroupMetadata = coordinator.groupMetadata(); @@ -3270,7 +3361,7 @@ public void onPartitionsRevoked(Collection partitions) { @Test public void testPrepareJoinAndRejoinAfterFailedRebalance() { final List partitions = singletonList(t1p); - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"), true)) { coordinator.ensureActiveGroup(); prepareOffsetCommitRequest(singletonMap(t1p, 100L), Errors.REBALANCE_IN_PROGRESS); @@ -3290,7 +3381,7 @@ public void testPrepareJoinAndRejoinAfterFailedRebalance() { MockTime time = new MockTime(1); // onJoinPrepare will be executed and onJoinComplete will not. - boolean res = coordinator.joinGroupIfNeeded(time.timer(2)); + boolean res = coordinator.joinGroupIfNeeded(time.timer(100)); assertFalse(res); assertFalse(client.hasPendingResponses()); @@ -3335,7 +3426,7 @@ public void testPrepareJoinAndRejoinAfterFailedRebalance() { @Test public void shouldLoseAllOwnedPartitionsBeforeRejoiningAfterDroppingOutOfTheGroup() { final List partitions = singletonList(t1p); - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"), true)) { final SystemTime realTime = new SystemTime(); coordinator.ensureActiveGroup(); @@ -3368,7 +3459,7 @@ public void shouldLoseAllOwnedPartitionsBeforeRejoiningAfterDroppingOutOfTheGrou @Test public void shouldLoseAllOwnedPartitionsBeforeRejoiningAfterResettingGenerationId() { final List partitions = singletonList(t1p); - try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"))) { + try (ConsumerCoordinator coordinator = prepareCoordinatorForCloseTest(true, false, Optional.of("group-id"), true)) { final SystemTime realTime = new SystemTime(); coordinator.ensureActiveGroup(); @@ -3435,7 +3526,11 @@ private void supportStableFlag(final short upperVersion, final boolean expectThr OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, leaderEpoch, metadata, Errors.NONE); - client.prepareResponse(new OffsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); + if (upperVersion < 8) { + client.prepareResponse(new OffsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); + } else { + client.prepareResponse(offsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); + } if (expectThrows) { assertThrows(UnsupportedVersionException.class, () -> coordinator.fetchCommittedOffsets(singleton(t1p), time.timer(Long.MAX_VALUE))); @@ -3462,7 +3557,8 @@ private void receiveFencedInstanceIdException() { private ConsumerCoordinator prepareCoordinatorForCloseTest(final boolean useGroupManagement, final boolean autoCommit, - final Optional groupInstanceId) { + final Optional groupInstanceId, + final boolean shouldPoll) { rebalanceConfig = buildRebalanceConfig(groupInstanceId); ConsumerCoordinator coordinator = buildCoordinator(rebalanceConfig, new Metrics(), @@ -3481,7 +3577,9 @@ private ConsumerCoordinator prepareCoordinatorForCloseTest(final boolean useGrou } subscriptions.seek(t1p, 100); - coordinator.poll(time.timer(Long.MAX_VALUE)); + if (shouldPoll) { + coordinator.poll(time.timer(Long.MAX_VALUE)); + } return coordinator; } @@ -3690,8 +3788,10 @@ private OffsetCommitResponse offsetCommitResponse(Map re return new OffsetCommitResponse(responseData); } - private OffsetFetchResponse offsetFetchResponse(Errors topLevelError) { - return new OffsetFetchResponse(topLevelError, Collections.emptyMap()); + private OffsetFetchResponse offsetFetchResponse(Errors error, Map responseData) { + return new OffsetFetchResponse(throttleMs, + singletonMap(groupId, error), + singletonMap(groupId, responseData)); } private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partitionLevelError, String metadata, long offset) { @@ -3701,7 +3801,7 @@ private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partit private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partitionLevelError, String metadata, long offset, Optional epoch) { OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, epoch, metadata, partitionLevelError); - return new OffsetFetchResponse(Errors.NONE, singletonMap(tp, data)); + return offsetFetchResponse(Errors.NONE, singletonMap(tp, data)); } private OffsetCommitCallback callback(final AtomicBoolean success) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 545bf24b9ec25..3588df2505c86 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -137,6 +137,7 @@ import static java.util.Arrays.asList; import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; import static java.util.Collections.singleton; import static java.util.Collections.singletonList; import static java.util.Collections.singletonMap; @@ -272,6 +273,45 @@ public void testFetchNormal() { } } + @Test + public void testInflightFetchOnPendingPartitions() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + assertEquals(1, fetcher.sendFetches()); + subscriptions.markPendingRevocation(singleton(tp0)); + + client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + consumerClient.poll(time.timer(0)); + assertNull(fetchedRecords().get(tp0)); + } + + @Test + public void testFetchingPendingPartitions() { + buildFetcher(); + + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 0); + + // normal fetch + assertEquals(1, fetcher.sendFetches()); + client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0)); + consumerClient.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchedRecords(); + assertEquals(4L, subscriptions.position(tp0).offset); // this is the next fetching position + + // mark partition unfetchable + subscriptions.markPendingRevocation(singleton(tp0)); + assertEquals(0, fetcher.sendFetches()); + consumerClient.poll(time.timer(0)); + assertFalse(fetcher.hasCompletedFetches()); + fetchedRecords(); + assertEquals(4L, subscriptions.position(tp0).offset); + } + @Test public void testFetchWithNoTopicId() { // Should work and default to using old request type. @@ -2283,6 +2323,37 @@ public void testRestOffsetsAuthorizationFailure() { assertEquals(5, subscriptions.position(tp0).offset); } + @Test + public void testFetchingPendingPartitionsBeforeAndAfterSubscriptionReset() { + buildFetcher(); + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 100); + subscriptions.seek(tp0, 100); + subscriptions.seek(tp0, 100); + assertEquals(100, subscriptions.position(tp0).offset); + + assertTrue(subscriptions.isFetchable(tp0)); // because tp is paused + + subscriptions.markPendingRevocation(singleton(tp0)); + fetcher.resetOffsetsIfNeeded(); + + // once a partition is marked pending, it should not be fetchable + assertFalse(subscriptions.isOffsetResetNeeded(tp0)); + assertFalse(subscriptions.isFetchable(tp0)); + assertTrue(subscriptions.hasValidPosition(tp0)); + assertEquals(100, subscriptions.position(tp0).offset); + + subscriptions.seek(tp0, 100); + assertEquals(100, subscriptions.position(tp0).offset); + + // reassignment should enable fetching of the same partition + subscriptions.unsubscribe(); + assignFromUser(singleton(tp0)); + subscriptions.seek(tp0, 100); + assertEquals(100, subscriptions.position(tp0).offset); + assertTrue(subscriptions.isFetchable(tp0)); + } + @Test public void testUpdateFetchPositionOfPausedPartitionsRequiringOffsetReset() { buildFetcher(); @@ -4624,12 +4695,12 @@ public void testPreferredReadReplica() { Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED, Duration.ofMinutes(5).toMillis()); subscriptions.assignFromUser(singleton(tp0)); - client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds)); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds, false)); subscriptions.seek(tp0, 0); // Node preferred replica before first fetch response Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); - assertEquals(selected.id(), -1); + assertEquals(-1, selected.id()); assertEquals(1, fetcher.sendFetches()); assertFalse(fetcher.hasCompletedFetches()); @@ -4643,9 +4714,9 @@ public void testPreferredReadReplica() { Map>> partitionRecords = fetchedRecords(); assertTrue(partitionRecords.containsKey(tp0)); - // verify + // Verify selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); - assertEquals(selected.id(), 1); + assertEquals(1, selected.id()); assertEquals(1, fetcher.sendFetches()); @@ -4658,7 +4729,110 @@ public void testPreferredReadReplica() { assertTrue(fetcher.hasCompletedFetches()); fetchedRecords(); selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); - assertEquals(selected.id(), -1); + assertEquals(-1, selected.id()); + } + + @Test + public void testFetchDisconnectedShouldClearPreferredReadReplica() { + buildFetcher(new MetricConfig(), OffsetResetStrategy.EARLIEST, new BytesDeserializer(), new BytesDeserializer(), + Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED, Duration.ofMinutes(5).toMillis()); + + subscriptions.assignFromUser(singleton(tp0)); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds, false)); + subscriptions.seek(tp0, 0); + assertEquals(1, fetcher.sendFetches()); + + // Set preferred read replica to node=1 + client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, + FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); + consumerClient.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchedRecords(); + + // Verify + Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(1, selected.id()); + assertEquals(1, fetcher.sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // Disconnect - preferred read replica should be cleared. + client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0), true); + + consumerClient.poll(time.timer(0)); + assertFalse(fetcher.hasCompletedFetches()); + fetchedRecords(); + selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(-1, selected.id()); + } + + @Test + public void testFetchDisconnectedShouldNotClearPreferredReadReplicaIfUnassigned() { + buildFetcher(new MetricConfig(), OffsetResetStrategy.EARLIEST, new BytesDeserializer(), new BytesDeserializer(), + Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED, Duration.ofMinutes(5).toMillis()); + + subscriptions.assignFromUser(singleton(tp0)); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds, false)); + subscriptions.seek(tp0, 0); + assertEquals(1, fetcher.sendFetches()); + + // Set preferred read replica to node=1 + client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, + FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); + consumerClient.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchedRecords(); + + // Verify + Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(1, selected.id()); + assertEquals(1, fetcher.sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // Disconnect and remove tp0 from assignment + client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0), true); + subscriptions.assignFromUser(emptySet()); + + // Preferred read replica should not be cleared + consumerClient.poll(time.timer(0)); + assertFalse(fetcher.hasCompletedFetches()); + fetchedRecords(); + selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(-1, selected.id()); + } + + @Test + public void testFetchErrorShouldClearPreferredReadReplica() { + buildFetcher(new MetricConfig(), OffsetResetStrategy.EARLIEST, new BytesDeserializer(), new BytesDeserializer(), + Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED, Duration.ofMinutes(5).toMillis()); + + subscriptions.assignFromUser(singleton(tp0)); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds, false)); + subscriptions.seek(tp0, 0); + assertEquals(1, fetcher.sendFetches()); + + // Set preferred read replica to node=1 + client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, + FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); + consumerClient.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchedRecords(); + + // Verify + Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(1, selected.id()); + assertEquals(1, fetcher.sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // Error - preferred read replica should be cleared. An actual error response will contain -1 as the + // preferred read replica. In the test we want to ensure that we are handling the error. + client.prepareResponse(fullFetchResponse(tidp0, MemoryRecords.EMPTY, Errors.NOT_LEADER_OR_FOLLOWER, -1L, + FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); + + consumerClient.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchedRecords(); + selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(-1, selected.id()); } @Test @@ -4667,7 +4841,7 @@ public void testPreferredReadReplicaOffsetError() { Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED, Duration.ofMinutes(5).toMillis()); subscriptions.assignFromUser(singleton(tp0)); - client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds)); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds, false)); subscriptions.seek(tp0, 0); @@ -5089,11 +5263,6 @@ private void buildFetcher(MetricConfig metricConfig, subscriptionState, logContext); } - private void buildFetcher(SubscriptionState subscriptionState, LogContext logContext) { - buildFetcher(new MetricConfig(), new ByteArrayDeserializer(), new ByteArrayDeserializer(), Integer.MAX_VALUE, - IsolationLevel.READ_UNCOMMITTED, Long.MAX_VALUE, subscriptionState, logContext); - } - private void buildFetcher(MetricConfig metricConfig, Deserializer keyDeserializer, Deserializer valueDeserializer, 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 d19234fe8a654..97c61616c75bc 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 @@ -256,6 +256,16 @@ public void partitionPause() { assertTrue(state.isFetchable(tp0)); } + @Test + public void testMarkingPartitionPending() { + state.assignFromUser(singleton(tp0)); + state.seek(tp0, 100); + assertTrue(state.isFetchable(tp0)); + state.markPendingRevocation(singleton(tp0)); + assertFalse(state.isFetchable(tp0)); + assertFalse(state.isPaused(tp0)); + } + @Test public void invalidPositionUpdate() { state.subscribe(singleton(topic), rebalanceListener); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index cf991de338bdd..9408172004eaf 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -1105,7 +1105,7 @@ public void testUniformBuiltInPartitioner() throws Exception { BuiltInPartitioner.mockRandom = () -> mockRandom.getAndAdd(1); long totalSize = 1024 * 1024; - int batchSize = 128; // note that this is also a "sticky" limit for the partitioner + int batchSize = 1024; // note that this is also a "sticky" limit for the partitioner RecordAccumulator accum = createTestRecordAccumulator(batchSize, totalSize, CompressionType.NONE, 0); // Set up callbacks so that we know what partition is chosen. @@ -1129,31 +1129,34 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { assertEquals(1, mockRandom.get()); // Produce large record, we should exceed "sticky" limit, but produce to this partition - // as we switch after the "sticky" limit is exceeded. The partition is switched after - // we produce. + // as we try to switch after the "sticky" limit is exceeded. The switch is disabled + // because of incomplete batch. byte[] largeValue = new byte[batchSize]; accum.append(topic, RecordMetadata.UNKNOWN_PARTITION, 0L, null, largeValue, Record.EMPTY_HEADERS, callbacks, maxBlockTimeMs, false, time.milliseconds(), cluster); assertEquals(partition1, partition.get()); - assertEquals(2, mockRandom.get()); + assertEquals(1, mockRandom.get()); - // Produce large record, we should switch to next partition. + // Produce large record, we should switch to next partition as we complete + // previous batch and exceeded sticky limit. accum.append(topic, RecordMetadata.UNKNOWN_PARTITION, 0L, null, largeValue, Record.EMPTY_HEADERS, callbacks, maxBlockTimeMs, false, time.milliseconds(), cluster); assertEquals(partition2, partition.get()); - assertEquals(3, mockRandom.get()); + assertEquals(2, mockRandom.get()); - // Produce large record, we should switch to next partition. + // Produce large record, we should switch to next partition as we complete + // previous batch and exceeded sticky limit. accum.append(topic, RecordMetadata.UNKNOWN_PARTITION, 0L, null, largeValue, Record.EMPTY_HEADERS, callbacks, maxBlockTimeMs, false, time.milliseconds(), cluster); assertEquals(partition3, partition.get()); - assertEquals(4, mockRandom.get()); + assertEquals(3, mockRandom.get()); - // Produce large record, we should switch to first partition again. + // Produce large record, we should switch to next partition as we complete + // previous batch and exceeded sticky limit. accum.append(topic, RecordMetadata.UNKNOWN_PARTITION, 0L, null, largeValue, Record.EMPTY_HEADERS, callbacks, maxBlockTimeMs, false, time.milliseconds(), cluster); assertEquals(partition1, partition.get()); - assertEquals(5, mockRandom.get()); + assertEquals(4, mockRandom.get()); } finally { BuiltInPartitioner.mockRandom = null; } @@ -1248,6 +1251,37 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { } } + @Test + public void testBuiltInPartitionerFractionalBatches() throws Exception { + // Test how we avoid creating fractional batches with high linger.ms (see + // BuiltInPartitioner.updatePartitionInfo). + long totalSize = 1024 * 1024; + int batchSize = 512; // note that this is also a "sticky" limit for the partitioner + int valSize = 32; + RecordAccumulator accum = createTestRecordAccumulator(batchSize, totalSize, CompressionType.NONE, 10); + byte[] value = new byte[valSize]; + + for (int c = 10; c-- > 0; ) { + // Produce about 2/3 of the batch size. + for (int recCount = batchSize * 2 / 3 / valSize; recCount-- > 0; ) { + accum.append(topic, RecordMetadata.UNKNOWN_PARTITION, 0, null, value, Record.EMPTY_HEADERS, + null, maxBlockTimeMs, false, time.milliseconds(), cluster); + } + + // Advance the time to make the batch ready. + time.sleep(10); + + // We should have one batch ready. + Set nodes = accum.ready(cluster, time.milliseconds()).readyNodes; + assertEquals(1, nodes.size(), "Should have 1 leader ready"); + List batches = accum.drain(cluster, nodes, Integer.MAX_VALUE, 0).entrySet().iterator().next().getValue(); + assertEquals(1, batches.size(), "Should have 1 batch ready"); + int actualBatchSize = batches.get(0).records().sizeInBytes(); + assertTrue(actualBatchSize > batchSize / 2, "Batch must be greater than half batch.size"); + assertTrue(actualBatchSize < batchSize, "Batch must be less than batch.size"); + } + } + private int prepareSplitBatches(RecordAccumulator accum, long seed, int recordSize, int numRecords) throws InterruptedException { Random random = new Random(); 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 b6bf9e6f4f1a7..ce9b80522076e 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 @@ -77,6 +77,8 @@ import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import java.nio.ByteBuffer; import java.util.Arrays; @@ -1678,6 +1680,61 @@ public void testMultipleAddPartitionsPerForOneProduce() throws InterruptedExcept assertTrue(secondResponseFuture.isDone()); } + @ParameterizedTest + @EnumSource(names = { + "UNKNOWN_TOPIC_OR_PARTITION", + "REQUEST_TIMED_OUT", + "COORDINATOR_LOAD_IN_PROGRESS", + "CONCURRENT_TRANSACTIONS" + }) + public void testRetriableErrors2(Errors error) { + // Ensure FindCoordinator retries. + TransactionalRequestResult result = transactionManager.initializeTransactions(); + prepareFindCoordinatorResponse(error, false, CoordinatorType.TRANSACTION, transactionalId); + prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); + runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); + assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); + + // Ensure InitPid retries. + prepareInitPidResponse(error, false, producerId, epoch); + prepareInitPidResponse(Errors.NONE, false, producerId, epoch); + runUntil(transactionManager::hasProducerId); + + result.await(); + transactionManager.beginTransaction(); + + // Ensure AddPartitionsToTxn retries. Since CONCURRENT_TRANSACTIONS is handled differently here, we substitute. + Errors addPartitionsToTxnError = error.equals(Errors.CONCURRENT_TRANSACTIONS) ? Errors.COORDINATOR_LOAD_IN_PROGRESS : error; + transactionManager.maybeAddPartition(tp0); + prepareAddPartitionsToTxnResponse(addPartitionsToTxnError, tp0, epoch, producerId); + prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, producerId); + runUntil(() -> transactionManager.transactionContainsPartition(tp0)); + + // Ensure txnOffsetCommit retries is tested in testRetriableErrorInTxnOffsetCommit. + + // Ensure EndTxn retries. + TransactionalRequestResult abortResult = transactionManager.beginCommit(); + prepareEndTxnResponse(error, TransactionResult.COMMIT, producerId, epoch); + prepareEndTxnResponse(Errors.NONE, TransactionResult.COMMIT, producerId, epoch); + runUntil(abortResult::isCompleted); + assertTrue(abortResult.isSuccessful()); + } + + @Test + public void testCoordinatorNotAvailable() { + // Ensure FindCoordinator with COORDINATOR_NOT_AVAILABLE error retries. + TransactionalRequestResult result = transactionManager.initializeTransactions(); + prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, false, CoordinatorType.TRANSACTION, transactionalId); + prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); + runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); + assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); + + prepareInitPidResponse(Errors.NONE, false, producerId, epoch); + runUntil(transactionManager::hasProducerId); + + result.await(); + } + @Test public void testProducerFencedExceptionInInitProducerId() { verifyProducerFencedForInitProducerId(Errors.PRODUCER_FENCED); diff --git a/clients/src/test/java/org/apache/kafka/common/message/SimpleArraysMessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/SimpleArraysMessageTest.java new file mode 100644 index 0000000000000..1b78adbb96227 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/message/SimpleArraysMessageTest.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.message; + +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.junit.jupiter.api.Test; + +import java.nio.ByteBuffer; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class SimpleArraysMessageTest { + @Test + public void testArrayBoundsChecking() { + // SimpleArraysMessageData takes 2 arrays + final ByteBuffer buf = ByteBuffer.wrap(new byte[] { + (byte) 0x7f, // Set size of first array to 126 which is larger than the size of this buffer + (byte) 0x00, (byte) 0x00, (byte) 0x00, (byte) 0x00, (byte) 0x00, (byte) 0x00, (byte) 0x00 + }); + final SimpleArraysMessageData out = new SimpleArraysMessageData(); + ByteBufferAccessor accessor = new ByteBufferAccessor(buf); + assertEquals("Tried to allocate a collection of size 126, but there are only 7 bytes remaining.", + assertThrows(RuntimeException.class, () -> out.read(accessor, (short) 2)).getMessage()); + } + + @Test + public void testArrayBoundsCheckingOtherArray() { + // SimpleArraysMessageData takes 2 arrays + final ByteBuffer buf = ByteBuffer.wrap(new byte[] { + (byte) 0x01, // Set size of first array to 0 + (byte) 0x7e, // Set size of second array to 125 which is larger than the size of this buffer + (byte) 0x00, (byte) 0x00, (byte) 0x00, (byte) 0x00, (byte) 0x00, (byte) 0x00 + }); + final SimpleArraysMessageData out = new SimpleArraysMessageData(); + ByteBufferAccessor accessor = new ByteBufferAccessor(buf); + assertEquals("Tried to allocate a collection of size 125, but there are only 6 bytes remaining.", + assertThrows(RuntimeException.class, () -> out.read(accessor, (short) 2)).getMessage()); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index 3dd114d9fd4b1..bc1fc5d9e5624 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -45,6 +45,7 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.internals.MetricsUtils; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.CumulativeSum; import org.apache.kafka.common.metrics.stats.Max; @@ -607,15 +608,15 @@ public void testRateWindowing() throws Exception { // Sleep for half the window. time.sleep(cfg.timeWindowMs() / 2); - // prior to any time passing - double elapsedSecs = (cfg.timeWindowMs() * (cfg.samples() - 1) + cfg.timeWindowMs() / 2) / 1000.0; + // prior to any time passing, elapsedSecs = sampleWindowSize * (total samples - half of final sample) + double elapsedSecs = MetricsUtils.convert(cfg.timeWindowMs(), TimeUnit.SECONDS) * (cfg.samples() - 0.5); KafkaMetric rateMetric = metrics.metrics().get(rateMetricName); KafkaMetric countRateMetric = metrics.metrics().get(countRateMetricName); assertEquals(sum / elapsedSecs, (Double) rateMetric.metricValue(), EPS, "Rate(0...2) = 2.666"); assertEquals(count / elapsedSecs, (Double) countRateMetric.metricValue(), EPS, "Count rate(0...2) = 0.02666"); assertEquals(elapsedSecs, - ((Rate) rateMetric.measurable()).windowSize(cfg, time.milliseconds()) / 1000, EPS, "Elapsed Time = 75 seconds"); + MetricsUtils.convert(((Rate) rateMetric.measurable()).windowSize(cfg, time.milliseconds()), TimeUnit.SECONDS), EPS, "Elapsed Time = 75 seconds"); assertEquals(sum, (Double) totalMetric.metricValue(), EPS); assertEquals(count, (Double) countTotalMetric.metricValue(), EPS); diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/stats/RateTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/stats/RateTest.java new file mode 100644 index 0000000000000..04c5ca1292f98 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/metrics/stats/RateTest.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.metrics.stats; + +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.internals.MetricsUtils; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; + +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +public class RateTest { + private static final double EPS = 0.000001; + private Rate r; + private Time timeClock; + + @BeforeEach + public void setup() { + r = new Rate(); + timeClock = new MockTime(); + } + + // Tests the scenario where the recording and measurement is done before the window for first sample finishes + // with no prior samples retained. + @ParameterizedTest + @CsvSource({"1,1", "1,11", "11,1", "11,11"}) + public void testRateWithNoPriorAvailableSamples(int numSample, int sampleWindowSizeSec) { + final MetricConfig config = new MetricConfig().samples(numSample).timeWindow(sampleWindowSizeSec, TimeUnit.SECONDS); + final double sampleValue = 50.0; + // record at beginning of the window + r.record(config, sampleValue, timeClock.milliseconds()); + // forward time till almost the end of window + final long measurementTime = TimeUnit.SECONDS.toMillis(sampleWindowSizeSec) - 1; + timeClock.sleep(measurementTime); + // calculate rate at almost the end of window + final double observedRate = r.measure(config, timeClock.milliseconds()); + assertFalse(Double.isNaN(observedRate)); + + // In a scenario where sufficient number of samples is not available yet, the rate calculation algorithm assumes + // presence of N-1 (where N = numSample) prior samples with sample values of 0. Hence, the window size for rate + // calculation accounts for N-1 prior samples + final int dummyPriorSamplesAssumedByAlgorithm = numSample - 1; + final double windowSize = MetricsUtils.convert(measurementTime, TimeUnit.SECONDS) + (dummyPriorSamplesAssumedByAlgorithm * sampleWindowSizeSec); + double expectedRatePerSec = sampleValue / windowSize; + assertEquals(expectedRatePerSec, observedRate, EPS); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ByteBufferAccessorTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ByteBufferAccessorTest.java new file mode 100644 index 0000000000000..6a0c6c2681c21 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/protocol/ByteBufferAccessorTest.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.protocol; + +import org.junit.jupiter.api.Test; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class ByteBufferAccessorTest { + @Test + public void testReadArray() { + ByteBuffer buf = ByteBuffer.allocate(1024); + ByteBufferAccessor accessor = new ByteBufferAccessor(buf); + final byte[] testArray = new byte[] {0x4b, 0x61, 0x46}; + accessor.writeByteArray(testArray); + accessor.writeInt(12345); + accessor.flip(); + final byte[] testArray2 = accessor.readArray(3); + assertArrayEquals(testArray, testArray2); + assertEquals(12345, accessor.readInt()); + assertEquals("Error reading byte array of 3 byte(s): only 0 byte(s) available", + assertThrows(RuntimeException.class, + () -> accessor.readArray(3)).getMessage()); + } + + @Test + public void testReadString() { + ByteBuffer buf = ByteBuffer.allocate(1024); + ByteBufferAccessor accessor = new ByteBufferAccessor(buf); + String testString = "ABC"; + final byte[] testArray = testString.getBytes(StandardCharsets.UTF_8); + accessor.writeByteArray(testArray); + accessor.flip(); + assertEquals("ABC", accessor.readString(3)); + assertEquals("Error reading byte array of 2 byte(s): only 0 byte(s) available", + assertThrows(RuntimeException.class, + () -> accessor.readString(2)).getMessage()); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/record/ControlRecordUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/record/ControlRecordUtilsTest.java index 657dc8005a23a..ed94a4e395843 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/ControlRecordUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/ControlRecordUtilsTest.java @@ -16,26 +16,48 @@ */ package org.apache.kafka.common.record; -import org.apache.kafka.common.message.LeaderChangeMessage; +import java.nio.ByteBuffer; +import java.util.Collections; import org.apache.kafka.common.message.LeaderChangeMessage.Voter; - +import org.apache.kafka.common.message.LeaderChangeMessage; +import org.apache.kafka.common.message.SnapshotFooterRecord; +import org.apache.kafka.common.message.SnapshotHeaderRecord; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.junit.jupiter.api.Test; - -import java.nio.ByteBuffer; -import java.util.Collections; - import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; public class ControlRecordUtilsTest { + @Test + public void testCurrentVersions() { + // If any of these asserts fail, please make sure that Kafka supports reading and + // writing the latest version for these records. + assertEquals( + LeaderChangeMessage.HIGHEST_SUPPORTED_VERSION, + ControlRecordUtils.LEADER_CHANGE_CURRENT_VERSION + ); + assertEquals( + SnapshotHeaderRecord.HIGHEST_SUPPORTED_VERSION, + ControlRecordUtils.SNAPSHOT_HEADER_CURRENT_VERSION + ); + assertEquals( + SnapshotFooterRecord.HIGHEST_SUPPORTED_VERSION, + ControlRecordUtils.SNAPSHOT_FOOTER_CURRENT_VERSION + ); + } + @Test public void testInvalidControlRecordType() { IllegalArgumentException thrown = assertThrows( - IllegalArgumentException.class, () -> testDeserializeRecord(ControlRecordType.COMMIT)); - assertEquals("Expected LEADER_CHANGE control record type(2), but found COMMIT", thrown.getMessage()); + IllegalArgumentException.class, + () -> testDeserializeRecord(ControlRecordType.COMMIT) + ); + assertEquals( + "Expected LEADER_CHANGE control record type(2), but found COMMIT", + thrown.getMessage() + ); } @Test @@ -47,9 +69,8 @@ private void testDeserializeRecord(ControlRecordType controlRecordType) { final int leaderId = 1; final int voterId = 2; LeaderChangeMessage data = new LeaderChangeMessage() - .setLeaderId(leaderId) - .setVoters(Collections.singletonList( - new Voter().setVoterId(voterId))); + .setLeaderId(leaderId) + .setVoters(Collections.singletonList(new Voter().setVoterId(voterId))); ByteBuffer valueBuffer = ByteBuffer.allocate(256); data.write(new ByteBufferAccessor(valueBuffer), new ObjectSerializationCache(), data.highestSupportedVersion()); diff --git a/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordTest.java b/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordTest.java index 49743d2320135..67212165fc35a 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordTest.java @@ -247,6 +247,20 @@ public void testInvalidNumHeaders() { buf.flip(); assertThrows(InvalidRecordException.class, () -> DefaultRecord.readFrom(buf, 0L, 0L, RecordBatch.NO_SEQUENCE, null)); + + ByteBuffer buf2 = ByteBuffer.allocate(sizeOfBodyInBytes + ByteUtils.sizeOfVarint(sizeOfBodyInBytes)); + ByteUtils.writeVarint(sizeOfBodyInBytes, buf2); + buf2.put(attributes); + ByteUtils.writeVarlong(timestampDelta, buf2); + ByteUtils.writeVarint(offsetDelta, buf2); + ByteUtils.writeVarint(-1, buf2); // null key + ByteUtils.writeVarint(-1, buf2); // null value + ByteUtils.writeVarint(sizeOfBodyInBytes, buf2); // more headers than remaining buffer size, not allowed + buf2.position(buf2.limit()); + + buf2.flip(); + assertThrows(InvalidRecordException.class, + () -> DefaultRecord.readFrom(buf2, 0L, 0L, RecordBatch.NO_SEQUENCE, null)); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ApiErrorTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ApiErrorTest.java index 8b0aa470becc9..bf352dbb4a871 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ApiErrorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ApiErrorTest.java @@ -41,10 +41,10 @@ public class ApiErrorTest { @ParameterizedTest @MethodSource("parameters") - public void fromThrowableShouldReturnCorrectError(Throwable t, Errors expectedErrors, String expectedMsg) { + public void fromThrowableShouldReturnCorrectError(Throwable t, Errors expectedError, String expectedMsg) { ApiError apiError = ApiError.fromThrowable(t); - assertEquals(apiError.error(), expectedErrors); - assertEquals(apiError.message(), expectedMsg); + assertEquals(expectedError, apiError.error()); + assertEquals(expectedMsg, apiError.message()); } private static Collection parameters() { 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 4415ff960aafb..254dea0430ece 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 @@ -16,22 +16,31 @@ */ package org.apache.kafka.common.requests; +import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.message.ApiVersionsResponseData; import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionCollection; import org.apache.kafka.common.message.CreateTopicsResponseData; +import org.apache.kafka.common.message.ProduceRequestData; +import org.apache.kafka.common.message.SaslAuthenticateRequestData; import org.apache.kafka.common.network.ClientInformation; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.junit.jupiter.api.Test; import java.net.InetAddress; +import java.net.UnknownHostException; import java.nio.ByteBuffer; +import java.util.Collections; 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 RequestContextTest { @@ -104,4 +113,78 @@ KafkaPrincipal.ANONYMOUS, new ListenerName("ssl"), SecurityProtocol.SASL_SSL, assertEquals(expectedResponse, parsedResponse.data()); } + @Test + public void testInvalidRequestForImplicitHashCollection() throws UnknownHostException { + short version = (short) 5; // choose a version with fixed length encoding, for simplicity + ByteBuffer corruptBuffer = produceRequest(version); + // corrupt the length of the topics array + corruptBuffer.putInt(8, (Integer.MAX_VALUE - 1) / 2); + + RequestHeader header = new RequestHeader(ApiKeys.PRODUCE, version, "console-producer", 3); + RequestContext context = new RequestContext(header, "0", InetAddress.getLocalHost(), + KafkaPrincipal.ANONYMOUS, new ListenerName("ssl"), SecurityProtocol.SASL_SSL, + ClientInformation.EMPTY, true); + + String msg = assertThrows(InvalidRequestException.class, + () -> context.parseRequest(corruptBuffer)).getCause().getMessage(); + assertEquals("Tried to allocate a collection of size 1073741823, but there are only 17 bytes remaining.", msg); + } + + @Test + public void testInvalidRequestForArrayList() throws UnknownHostException { + short version = (short) 5; // choose a version with fixed length encoding, for simplicity + ByteBuffer corruptBuffer = produceRequest(version); + // corrupt the length of the partitions array + corruptBuffer.putInt(17, Integer.MAX_VALUE); + + RequestHeader header = new RequestHeader(ApiKeys.PRODUCE, version, "console-producer", 3); + RequestContext context = new RequestContext(header, "0", InetAddress.getLocalHost(), + KafkaPrincipal.ANONYMOUS, new ListenerName("ssl"), SecurityProtocol.SASL_SSL, + ClientInformation.EMPTY, true); + + String msg = assertThrows(InvalidRequestException.class, + () -> context.parseRequest(corruptBuffer)).getCause().getMessage(); + assertEquals( + "Tried to allocate a collection of size 2147483647, but there are only 8 bytes remaining.", msg); + } + + private ByteBuffer produceRequest(short version) { + ProduceRequestData data = new ProduceRequestData() + .setAcks((short) -1) + .setTimeoutMs(1); + data.topicData().add( + new ProduceRequestData.TopicProduceData() + .setName("foo") + .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() + .setIndex(42)))); + + return serialize(version, data); + } + + private ByteBuffer serialize(short version, ApiMessage data) { + ObjectSerializationCache cache = new ObjectSerializationCache(); + data.size(cache, version); + ByteBuffer buffer = ByteBuffer.allocate(1024); + data.write(new ByteBufferAccessor(buffer), cache, version); + buffer.flip(); + return buffer; + } + + @Test + public void testInvalidRequestForByteArray() throws UnknownHostException { + short version = (short) 1; // choose a version with fixed length encoding, for simplicity + ByteBuffer corruptBuffer = serialize(version, new SaslAuthenticateRequestData().setAuthBytes(new byte[0])); + // corrupt the length of the bytes array + corruptBuffer.putInt(0, Integer.MAX_VALUE); + + RequestHeader header = new RequestHeader(ApiKeys.SASL_AUTHENTICATE, version, "console-producer", 1); + RequestContext context = new RequestContext(header, "0", InetAddress.getLocalHost(), + KafkaPrincipal.ANONYMOUS, new ListenerName("ssl"), SecurityProtocol.SASL_SSL, + ClientInformation.EMPTY, true); + + String msg = assertThrows(InvalidRequestException.class, + () -> context.parseRequest(corruptBuffer)).getCause().getMessage(); + assertEquals("Error reading byte array of 2147483647 byte(s): only 0 byte(s) available", msg); + } + } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index b6df4c44d6392..3fa063fba186a 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -210,6 +210,7 @@ import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ObjectSerializationCache; +import org.apache.kafka.common.protocol.types.RawTaggedField; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; import org.apache.kafka.common.quota.ClientQuotaFilter; @@ -231,6 +232,7 @@ import org.apache.kafka.common.security.token.delegation.TokenInformation; import org.apache.kafka.common.utils.SecurityUtils; import org.apache.kafka.common.utils.Utils; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.nio.BufferUnderflowException; @@ -269,6 +271,7 @@ import static org.apache.kafka.common.protocol.ApiKeys.METADATA; import static org.apache.kafka.common.protocol.ApiKeys.OFFSET_FETCH; import static org.apache.kafka.common.protocol.ApiKeys.PRODUCE; +import static org.apache.kafka.common.protocol.ApiKeys.SASL_AUTHENTICATE; import static org.apache.kafka.common.protocol.ApiKeys.STOP_REPLICA; import static org.apache.kafka.common.protocol.ApiKeys.SYNC_GROUP; import static org.apache.kafka.common.protocol.ApiKeys.UPDATE_METADATA; @@ -1753,10 +1756,8 @@ private JoinGroupResponse createJoinGroupResponse(short version) { for (int i = 0; i < 2; i++) { JoinGroupResponseMember member = new JoinGroupResponseData.JoinGroupResponseMember() .setMemberId("consumer" + i) - .setMetadata(new byte[0]); - - if (version >= 5) - member.setGroupInstanceId("instance" + i); + .setMetadata(new byte[0]) + .setGroupInstanceId("instance" + i); members.add(member); } @@ -3360,4 +3361,92 @@ private ListTransactionsResponse createListTransactionsResponse() { return new ListTransactionsResponse(response); } + @Test + public void testInvalidSaslHandShakeRequest() { + AbstractRequest request = new SaslHandshakeRequest.Builder( + new SaslHandshakeRequestData().setMechanism("PLAIN")).build(); + ByteBuffer serializedBytes = request.serialize(); + // corrupt the length of the sasl mechanism string + serializedBytes.putShort(0, Short.MAX_VALUE); + + String msg = assertThrows(RuntimeException.class, () -> AbstractRequest. + parseRequest(request.apiKey(), request.version(), serializedBytes)).getMessage(); + assertEquals("Error reading byte array of 32767 byte(s): only 5 byte(s) available", msg); + } + + @Test + public void testInvalidSaslAuthenticateRequest() { + short version = (short) 1; // choose a version with fixed length encoding, for simplicity + byte[] b = new byte[] { + 0x11, 0x1f, 0x15, 0x2c, + 0x5e, 0x2a, 0x20, 0x26, + 0x6c, 0x39, 0x45, 0x1f, + 0x25, 0x1c, 0x2d, 0x25, + 0x43, 0x2a, 0x11, 0x76 + }; + SaslAuthenticateRequestData data = new SaslAuthenticateRequestData().setAuthBytes(b); + AbstractRequest request = new SaslAuthenticateRequest(data, version); + ByteBuffer serializedBytes = request.serialize(); + + // corrupt the length of the bytes array + serializedBytes.putInt(0, Integer.MAX_VALUE); + + String msg = assertThrows(RuntimeException.class, () -> AbstractRequest. + parseRequest(request.apiKey(), request.version(), serializedBytes)).getMessage(); + assertEquals("Error reading byte array of 2147483647 byte(s): only 20 byte(s) available", msg); + } + + @Test + public void testValidTaggedFieldsWithSaslAuthenticateRequest() { + byte[] byteArray = new byte[11]; + ByteBufferAccessor accessor = new ByteBufferAccessor(ByteBuffer.wrap(byteArray)); + + //construct a SASL_AUTHENTICATE request + byte[] authBytes = "test".getBytes(StandardCharsets.UTF_8); + accessor.writeUnsignedVarint(authBytes.length + 1); + accessor.writeByteArray(authBytes); + + //write total numbers of tags + accessor.writeUnsignedVarint(1); + + //write first tag + RawTaggedField taggedField = new RawTaggedField(1, new byte[] {0x1, 0x2, 0x3}); + accessor.writeUnsignedVarint(taggedField.tag()); + accessor.writeUnsignedVarint(taggedField.size()); + accessor.writeByteArray(taggedField.data()); + + accessor.flip(); + + SaslAuthenticateRequest saslAuthenticateRequest = (SaslAuthenticateRequest) AbstractRequest. + parseRequest(SASL_AUTHENTICATE, SASL_AUTHENTICATE.latestVersion(), accessor.buffer()).request; + Assertions.assertArrayEquals(authBytes, saslAuthenticateRequest.data().authBytes()); + assertEquals(1, saslAuthenticateRequest.data().unknownTaggedFields().size()); + assertEquals(taggedField, saslAuthenticateRequest.data().unknownTaggedFields().get(0)); + } + + @Test + public void testInvalidTaggedFieldsWithSaslAuthenticateRequest() { + byte[] byteArray = new byte[13]; + ByteBufferAccessor accessor = new ByteBufferAccessor(ByteBuffer.wrap(byteArray)); + + //construct a SASL_AUTHENTICATE request + byte[] authBytes = "test".getBytes(StandardCharsets.UTF_8); + accessor.writeUnsignedVarint(authBytes.length + 1); + accessor.writeByteArray(authBytes); + + //write total numbers of tags + accessor.writeUnsignedVarint(1); + + //write first tag + RawTaggedField taggedField = new RawTaggedField(1, new byte[] {0x1, 0x2, 0x3}); + accessor.writeUnsignedVarint(taggedField.tag()); + accessor.writeUnsignedVarint(Short.MAX_VALUE); // set wrong size for tagged field + accessor.writeByteArray(taggedField.data()); + + accessor.flip(); + + String msg = assertThrows(RuntimeException.class, () -> AbstractRequest. + parseRequest(SASL_AUTHENTICATE, SASL_AUTHENTICATE.latestVersion(), accessor.buffer())).getMessage(); + assertEquals("Error reading byte array of 32767 byte(s): only 3 byte(s) available", msg); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java index d50e1b90913bb..ebc74c807c4bf 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java @@ -37,6 +37,7 @@ import java.util.Map; import java.util.Optional; import java.util.function.Function; +import java.util.stream.Collectors; public class RequestTestUtils { @@ -177,6 +178,16 @@ public static MetadataResponse metadataUpdateWithIds(final int numNodes, topicIds); } + public static MetadataResponse metadataUpdateWithIds(final int numNodes, + final Map topicPartitionCounts, + final Function epochSupplier, + final Map topicIds, + final Boolean leaderOnly) { + return metadataUpdateWith("kafka-cluster", numNodes, Collections.emptyMap(), + topicPartitionCounts, epochSupplier, MetadataResponse.PartitionMetadata::new, ApiKeys.METADATA.latestVersion(), + topicIds, leaderOnly); + } + public static MetadataResponse metadataUpdateWithIds(final String clusterId, final int numNodes, final Map topicErrors, @@ -195,6 +206,20 @@ public static MetadataResponse metadataUpdateWith(final String clusterId, final PartitionMetadataSupplier partitionSupplier, final short responseVersion, final Map topicIds) { + return metadataUpdateWith(clusterId, numNodes, topicErrors, + topicPartitionCounts, epochSupplier, partitionSupplier, + responseVersion, topicIds, true); + } + + public static MetadataResponse metadataUpdateWith(final String clusterId, + final int numNodes, + final Map topicErrors, + final Map topicPartitionCounts, + final Function epochSupplier, + final PartitionMetadataSupplier partitionSupplier, + final short responseVersion, + final Map topicIds, + final Boolean leaderOnly) { final List nodes = new ArrayList<>(numNodes); for (int i = 0; i < numNodes; i++) nodes.add(new Node(i, "localhost", 1969 + i)); @@ -208,10 +233,10 @@ public static MetadataResponse metadataUpdateWith(final String clusterId, for (int i = 0; i < numPartitions; i++) { TopicPartition tp = new TopicPartition(topic, i); Node leader = nodes.get(i % nodes.size()); - List replicaIds = Collections.singletonList(leader.id()); + List replicaIds = leaderOnly ? Collections.singletonList(leader.id()) : nodes.stream().map(Node::id).collect(Collectors.toList()); partitionMetadata.add(partitionSupplier.supply( Errors.NONE, tp, Optional.of(leader.id()), Optional.ofNullable(epochSupplier.apply(tp)), - replicaIds, replicaIds, replicaIds)); + replicaIds, replicaIds, Collections.emptyList())); } topicMetadata.add(new MetadataResponse.TopicMetadata(Errors.NONE, topic, topicIds.getOrDefault(topic, Uuid.ZERO_UUID), diff --git a/clients/src/test/java/org/apache/kafka/common/security/SaslExtensionsTest.java b/clients/src/test/java/org/apache/kafka/common/security/SaslExtensionsTest.java index 9acb78cf3ef50..085baf70d2a2a 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/SaslExtensionsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/SaslExtensionsTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.common.security; +import java.util.Collections; import org.apache.kafka.common.security.auth.SaslExtensions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -23,6 +24,7 @@ import java.util.HashMap; import java.util.Map; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -50,4 +52,30 @@ public void testCannotAddValueToMapReferenceAndGetFromExtensions() { this.map.put("hello", "42"); assertNull(extensions.map().get("hello")); } + + /** + * Tests that even when using the same underlying values in the map, two {@link SaslExtensions} + * are considered unique. + * + * @see SaslExtensions class-level documentation + */ + @Test + public void testExtensionsWithEqualValuesAreUnique() { + // If the maps are distinct objects but have the same underlying values, the SaslExtension + // objects should still be unique. + assertNotEquals(new SaslExtensions(Collections.singletonMap("key", "value")), + new SaslExtensions(Collections.singletonMap("key", "value")), + "SaslExtensions with unique maps should be unique"); + + // If the maps are the same object (with the same underlying values), the SaslExtension + // objects should still be unique. + assertNotEquals(new SaslExtensions(map), + new SaslExtensions(map), + "SaslExtensions with duplicate maps should be unique"); + + // If the maps are empty, the SaslExtension objects should still be unique. + assertNotEquals(SaslExtensions.empty(), + SaslExtensions.empty(), + "SaslExtensions returned from SaslExtensions.empty() should be unique"); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/TestSecurityConfig.java b/clients/src/test/java/org/apache/kafka/common/security/TestSecurityConfig.java index 07cbb7856dded..197151f5fbc20 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/TestSecurityConfig.java +++ b/clients/src/test/java/org/apache/kafka/common/security/TestSecurityConfig.java @@ -38,6 +38,8 @@ public class TestSecurityConfig extends AbstractConfig { null, Importance.MEDIUM, BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DOC) .define(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS, Type.LONG, 0L, Importance.MEDIUM, BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_DOC) + .define(BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG, Type.INT, BrokerSecurityConfigs.DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE, + Importance.LOW, BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_DOC) .withClientSslSupport() .withClientSaslSupport(); 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 988a0f2823213..40a27935f3f74 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 @@ -212,6 +212,52 @@ public void testValidSaslPlainOverPlaintext() throws Exception { checkAuthenticationAndReauthentication(securityProtocol, node); } + /** + * Test SASL/PLAIN with sasl.authentication.max.receive.size config + */ + @Test + public void testSaslAuthenticationMaxReceiveSize() throws Exception { + SecurityProtocol securityProtocol = SecurityProtocol.SASL_PLAINTEXT; + configureMechanisms("PLAIN", Collections.singletonList("PLAIN")); + + // test auth with 1KB receive size + saslServerConfigs.put(BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG, "1024"); + server = createEchoServer(securityProtocol); + + // test valid sasl authentication + String node1 = "valid"; + checkAuthenticationAndReauthentication(securityProtocol, node1); + + // test with handshake request with large mechanism string + byte[] bytes = new byte[1024]; + new Random().nextBytes(bytes); + String mechanism = new String(bytes, StandardCharsets.UTF_8); + String node2 = "invalid1"; + createClientConnection(SecurityProtocol.PLAINTEXT, node2); + SaslHandshakeRequest handshakeRequest = buildSaslHandshakeRequest(mechanism, ApiKeys.SASL_HANDSHAKE.latestVersion()); + RequestHeader header = new RequestHeader(ApiKeys.SASL_HANDSHAKE, handshakeRequest.version(), "someclient", nextCorrelationId++); + NetworkSend send = new NetworkSend(node2, handshakeRequest.toSend(header)); + selector.send(send); + //we will get exception in server and connection gets closed. + NetworkTestUtils.waitForChannelClose(selector, node2, ChannelState.READY.state()); + selector.close(); + + String node3 = "invalid2"; + createClientConnection(SecurityProtocol.PLAINTEXT, node3); + sendHandshakeRequestReceiveResponse(node3, ApiKeys.SASL_HANDSHAKE.latestVersion()); + + // test with sasl authenticate request with large auth_byes string + String authString = "\u0000" + TestJaasConfig.USERNAME + "\u0000" + new String(bytes, StandardCharsets.UTF_8); + ByteBuffer authBuf = ByteBuffer.wrap(Utils.utf8(authString)); + SaslAuthenticateRequestData data = new SaslAuthenticateRequestData().setAuthBytes(authBuf.array()); + SaslAuthenticateRequest request = new SaslAuthenticateRequest.Builder(data).build(); + header = new RequestHeader(ApiKeys.SASL_AUTHENTICATE, request.version(), "someclient", nextCorrelationId++); + send = new NetworkSend(node3, request.toSend(header)); + selector.send(send); + NetworkTestUtils.waitForChannelClose(selector, node3, ChannelState.READY.state()); + server.verifyAuthenticationMetrics(1, 2); + } + /** * Tests that SASL/PLAIN clients with invalid password fail authentication. */ diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java index 50696ecf0512d..b0dec3e522ad8 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.config.internals.BrokerSecurityConfigs; import org.apache.kafka.common.errors.IllegalSaslStateException; +import org.apache.kafka.common.errors.SaslAuthenticationException; import org.apache.kafka.common.message.ApiMessageType; import org.apache.kafka.common.message.SaslAuthenticateRequestData; import org.apache.kafka.common.message.SaslHandshakeRequestData; @@ -25,7 +26,6 @@ import org.apache.kafka.common.network.ChannelMetadataRegistry; import org.apache.kafka.common.network.ClientInformation; import org.apache.kafka.common.network.DefaultChannelMetadataRegistry; -import org.apache.kafka.common.network.InvalidReceiveException; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.network.TransportLayer; import org.apache.kafka.common.protocol.ApiKeys; @@ -96,10 +96,10 @@ public void testOversizeRequest() throws IOException { SCRAM_SHA_256.mechanismName(), new DefaultChannelMetadataRegistry()); when(transportLayer.read(any(ByteBuffer.class))).then(invocation -> { - invocation.getArgument(0).putInt(SaslServerAuthenticator.MAX_RECEIVE_SIZE + 1); + invocation.getArgument(0).putInt(BrokerSecurityConfigs.DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE + 1); return 4; }); - assertThrows(InvalidReceiveException.class, authenticator::authenticate); + assertThrows(SaslAuthenticationException.class, authenticator::authenticate); verify(transportLayer).read(any(ByteBuffer.class)); } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginModuleTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginModuleTest.java index ea03ec5bfa339..0dabeab1f433f 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginModuleTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerLoginModuleTest.java @@ -127,8 +127,8 @@ public void login1Commit1Login2Commit2Logout1Login3Commit3Logout2() throws Login // Create callback handler OAuthBearerToken[] tokens = new OAuthBearerToken[] {mock(OAuthBearerToken.class), mock(OAuthBearerToken.class), mock(OAuthBearerToken.class)}; - SaslExtensions[] extensions = new SaslExtensions[] {mock(SaslExtensions.class), - mock(SaslExtensions.class), mock(SaslExtensions.class)}; + SaslExtensions[] extensions = new SaslExtensions[] {saslExtensions(), + saslExtensions(), saslExtensions()}; TestCallbackHandler testTokenCallbackHandler = new TestCallbackHandler(tokens, extensions); // Create login modules @@ -208,7 +208,6 @@ public void login1Commit1Login2Commit2Logout1Login3Commit3Logout2() throws Login assertSame(extensions[2], publicCredentials.iterator().next()); verifyNoInteractions((Object[]) tokens); - verifyNoInteractions((Object[]) extensions); } @Test @@ -224,8 +223,8 @@ public void login1Commit1Logout1Login2Commit2Logout2() throws LoginException { // Create callback handler OAuthBearerToken[] tokens = new OAuthBearerToken[] {mock(OAuthBearerToken.class), mock(OAuthBearerToken.class)}; - SaslExtensions[] extensions = new SaslExtensions[] {mock(SaslExtensions.class), - mock(SaslExtensions.class)}; + SaslExtensions[] extensions = new SaslExtensions[] {saslExtensions(), + saslExtensions()}; TestCallbackHandler testTokenCallbackHandler = new TestCallbackHandler(tokens, extensions); // Create login modules @@ -270,7 +269,6 @@ public void login1Commit1Logout1Login2Commit2Logout2() throws LoginException { assertEquals(0, publicCredentials.size()); verifyNoInteractions((Object[]) tokens); - verifyNoInteractions((Object[]) extensions); } @Test @@ -285,8 +283,8 @@ public void loginAbortLoginCommitLogout() throws LoginException { // Create callback handler OAuthBearerToken[] tokens = new OAuthBearerToken[] {mock(OAuthBearerToken.class), mock(OAuthBearerToken.class)}; - SaslExtensions[] extensions = new SaslExtensions[] {mock(SaslExtensions.class), - mock(SaslExtensions.class)}; + SaslExtensions[] extensions = new SaslExtensions[] {saslExtensions(), + saslExtensions()}; TestCallbackHandler testTokenCallbackHandler = new TestCallbackHandler(tokens, extensions); // Create login module @@ -322,7 +320,6 @@ public void loginAbortLoginCommitLogout() throws LoginException { assertEquals(0, publicCredentials.size()); verifyNoInteractions((Object[]) tokens); - verifyNoInteractions((Object[]) extensions); } @Test @@ -338,8 +335,8 @@ public void login1Commit1Login2Abort2Login3Commit3Logout3() throws LoginExceptio // Create callback handler OAuthBearerToken[] tokens = new OAuthBearerToken[] {mock(OAuthBearerToken.class), mock(OAuthBearerToken.class), mock(OAuthBearerToken.class)}; - SaslExtensions[] extensions = new SaslExtensions[] {mock(SaslExtensions.class), - mock(SaslExtensions.class), mock(SaslExtensions.class)}; + SaslExtensions[] extensions = new SaslExtensions[] {saslExtensions(), saslExtensions(), + saslExtensions()}; TestCallbackHandler testTokenCallbackHandler = new TestCallbackHandler(tokens, extensions); // Create login modules @@ -406,7 +403,6 @@ public void login1Commit1Login2Abort2Login3Commit3Logout3() throws LoginExceptio assertSame(extensions[2], publicCredentials.iterator().next()); verifyNoInteractions((Object[]) tokens); - verifyNoInteractions((Object[]) extensions); } /** @@ -436,4 +432,21 @@ public void commitDoesNotThrowOnUnsupportedExtensionsCallback() throws LoginExce verifyNoInteractions((Object[]) tokens); } + + /** + * We don't want to use mocks for our tests as we need to make sure to test + * {@link SaslExtensions}' {@link SaslExtensions#equals(Object)} and + * {@link SaslExtensions#hashCode()} methods. + * + *

+ * + * We need to make distinct calls to this method (vs. caching the result and reusing it + * multiple times) because we need to ensure the {@link SaslExtensions} instances are unique. + * This properly mimics the behavior that is used during the token refresh logic. + * + * @return Unique, newly-created {@link SaslExtensions} instance + */ + private SaslExtensions saslExtensions() { + return SaslExtensions.empty(); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetrieverTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetrieverTest.java index de3b463447009..8a89091dc3409 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetrieverTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/secured/HttpAccessTokenRetrieverTest.java @@ -20,6 +20,7 @@ import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; 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.when; @@ -32,6 +33,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.net.HttpURLConnection; +import java.nio.charset.StandardCharsets; import java.util.Base64; import java.util.Random; import org.apache.kafka.common.utils.Utils; @@ -61,6 +63,60 @@ public void testErrorReadingResponse() throws IOException { assertThrows(IOException.class, () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null)); } + @Test + public void testErrorResponseUnretryableCode() throws IOException { + HttpURLConnection mockedCon = createHttpURLConnection("dummy"); + when(mockedCon.getInputStream()).thenThrow(new IOException("Can't read")); + when(mockedCon.getErrorStream()).thenReturn(new ByteArrayInputStream( + "{\"error\":\"some_arg\", \"error_description\":\"some problem with arg\"}" + .getBytes(StandardCharsets.UTF_8))); + when(mockedCon.getResponseCode()).thenReturn(HttpURLConnection.HTTP_BAD_REQUEST); + UnretryableException ioe = assertThrows(UnretryableException.class, + () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null)); + assertTrue(ioe.getMessage().contains("{\"some_arg\" - \"some problem with arg\"}")); + } + + @Test + public void testErrorResponseRetryableCode() throws IOException { + HttpURLConnection mockedCon = createHttpURLConnection("dummy"); + when(mockedCon.getInputStream()).thenThrow(new IOException("Can't read")); + when(mockedCon.getErrorStream()).thenReturn(new ByteArrayInputStream( + "{\"error\":\"some_arg\", \"error_description\":\"some problem with arg\"}" + .getBytes(StandardCharsets.UTF_8))); + when(mockedCon.getResponseCode()).thenReturn(HttpURLConnection.HTTP_INTERNAL_ERROR); + IOException ioe = assertThrows(IOException.class, + () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null)); + assertTrue(ioe.getMessage().contains("{\"some_arg\" - \"some problem with arg\"}")); + + // error response body has different keys + when(mockedCon.getErrorStream()).thenReturn(new ByteArrayInputStream( + "{\"errorCode\":\"some_arg\", \"errorSummary\":\"some problem with arg\"}" + .getBytes(StandardCharsets.UTF_8))); + ioe = assertThrows(IOException.class, + () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null)); + assertTrue(ioe.getMessage().contains("{\"some_arg\" - \"some problem with arg\"}")); + + // error response is valid json but unknown keys + when(mockedCon.getErrorStream()).thenReturn(new ByteArrayInputStream( + "{\"err\":\"some_arg\", \"err_des\":\"some problem with arg\"}" + .getBytes(StandardCharsets.UTF_8))); + ioe = assertThrows(IOException.class, + () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null)); + assertTrue(ioe.getMessage().contains("{\"err\":\"some_arg\", \"err_des\":\"some problem with arg\"}")); + } + + @Test + public void testErrorResponseIsInvalidJson() throws IOException { + HttpURLConnection mockedCon = createHttpURLConnection("dummy"); + when(mockedCon.getInputStream()).thenThrow(new IOException("Can't read")); + when(mockedCon.getErrorStream()).thenReturn(new ByteArrayInputStream( + "non json error output".getBytes(StandardCharsets.UTF_8))); + when(mockedCon.getResponseCode()).thenReturn(HttpURLConnection.HTTP_INTERNAL_ERROR); + IOException ioe = assertThrows(IOException.class, + () -> HttpAccessTokenRetriever.post(mockedCon, null, null, null, null)); + assertTrue(ioe.getMessage().contains("{non json error output}")); + } + @Test public void testCopy() throws IOException { byte[] expected = new byte[4096 + 1]; @@ -116,10 +172,19 @@ public void testParseAccessTokenInvalidJson() { @Test public void testFormatAuthorizationHeader() throws IOException { - String expected = "Basic " + Base64.getUrlEncoder().encodeToString(Utils.utf8("id:secret")); + assertAuthorizationHeader("id", "secret"); + } - String actual = HttpAccessTokenRetriever.formatAuthorizationHeader("id", "secret"); - assertEquals(expected, actual); + @Test + public void testFormatAuthorizationHeaderEncoding() throws IOException { + // See KAFKA-14496 + assertAuthorizationHeader("SOME_RANDOM_LONG_USER_01234", "9Q|0`8i~ute-n9ksjLWb\\50\"AX@UUED5E"); + } + + private void assertAuthorizationHeader(String clientId, String clientSecret) throws IOException { + String expected = "Basic " + Base64.getEncoder().encodeToString(Utils.utf8(clientId + ":" + clientSecret)); + String actual = HttpAccessTokenRetriever.formatAuthorizationHeader(clientId, clientSecret); + assertEquals(expected, actual, String.format("Expected the HTTP Authorization header generated for client ID \"%s\" and client secret \"%s\" to match", clientId, clientSecret)); } @Test diff --git a/clients/src/test/resources/common/message/SimpleArraysMessage.json b/clients/src/test/resources/common/message/SimpleArraysMessage.json new file mode 100644 index 0000000000000..76dc283b6a747 --- /dev/null +++ b/clients/src/test/resources/common/message/SimpleArraysMessage.json @@ -0,0 +1,29 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +{ + "name": "SimpleArraysMessage", + "type": "header", + "validVersions": "0-2", + "flexibleVersions": "1+", + "fields": [ + { "name": "Goats", "type": "[]StructArray", "versions": "1+", + "fields": [ + { "name": "Color", "type": "int8", "versions": "1+"}, + { "name": "Name", "type": "string", "versions": "2+"} + ] + }, + { "name": "Sheep", "type": "[]int32", "versions": "0+" } + ] +} diff --git a/config/kraft/README.md b/config/kraft/README.md deleted file mode 100644 index d280f87958be1..0000000000000 --- a/config/kraft/README.md +++ /dev/null @@ -1,177 +0,0 @@ -KRaft (aka KIP-500) mode Preview Release -========================================================= - -# Introduction -It is now possible to run Apache Kafka without Apache ZooKeeper! We call this the [Kafka Raft metadata mode](https://cwiki.apache.org/confluence/display/KAFKA/KIP-500%3A+Replace+ZooKeeper+with+a+Self-Managed+Metadata+Quorum), typically shortened to `KRaft mode`. -`KRaft` is intended to be pronounced like `craft` (as in `craftsmanship`). It is currently *PREVIEW AND SHOULD NOT BE USED IN PRODUCTION*, but it -is available for testing in the Kafka 3.1 release. - -When the Kafka cluster is in KRaft mode, it does not store its metadata in ZooKeeper. In fact, you do not have to run ZooKeeper at all, because it stores its metadata in a KRaft quorum of controller nodes. - -KRaft mode has many benefits -- some obvious, and some not so obvious. Clearly, it is nice to manage and configure one service rather than two services. In addition, you can now run a single process Kafka cluster. -Most important of all, KRaft mode is more scalable. We expect to be able to [support many more topics and partitions](https://www.confluent.io/kafka-summit-san-francisco-2019/kafka-needs-no-keeper/) in this mode. - -# Quickstart - -## Warning -KRaft mode in Kafka 3.1 is provided for testing only, *NOT* for production. We do not yet support upgrading existing ZooKeeper-based Kafka clusters into this mode. -There may be bugs, including serious ones. You should *assume that your data could be lost at any time* if you try the preview release of KRaft mode. - -## Generate a cluster ID -The first step is to generate an ID for your new cluster, using the kafka-storage tool: - -~~~~ -$ ./bin/kafka-storage.sh random-uuid -xtzWWN4bTjitpL3kfd9s5g -~~~~ - -## Format Storage Directories -The next step is to format your storage directories. If you are running in single-node mode, you can do this with one command: - -~~~~ -$ ./bin/kafka-storage.sh format -t -c ./config/kraft/server.properties -Formatting /tmp/kraft-combined-logs -~~~~ - -If you are using multiple nodes, then you should run the format command on each node. Be sure to use the same cluster ID for each one. - -This example configures the node as both a broker and controller (i.e. `process.roles=broker,controller`). It is also possible to run the broker and controller nodes separately. -Please see [here](https://github.com/apache/kafka/blob/trunk/config/kraft/broker.properties) and [here](https://github.com/apache/kafka/blob/trunk/config/kraft/controller.properties) for example configurations. - -## Start the Kafka Server -Finally, you are ready to start the Kafka server on each node. - -~~~~ -$ ./bin/kafka-server-start.sh ./config/kraft/server.properties -[2021-02-26 15:37:11,071] INFO Registered kafka:type=kafka.Log4jController MBean (kafka.utils.Log4jControllerRegistration$) -[2021-02-26 15:37:11,294] INFO Setting -D jdk.tls.rejectClientInitiatedRenegotiation=true to disable client-initiated TLS renegotiation (org.apache.zookeeper.common.X509Util) -[2021-02-26 15:37:11,466] INFO [Log partition=__cluster_metadata-0, dir=/tmp/kraft-combined-logs] Loading producer state till offset 0 with message format version 2 (kafka.log.Log) -[2021-02-26 15:37:11,509] INFO [raft-expiration-reaper]: Starting (kafka.raft.TimingWheelExpirationService$ExpiredOperationReaper) -[2021-02-26 15:37:11,640] INFO [RaftManager nodeId=1] Completed transition to Unattached(epoch=0, voters=[1], electionTimeoutMs=9037) (org.apache.kafka.raft.QuorumState) -... -~~~~ - -Just like with a ZooKeeper based broker, you can connect to port 9092 (or whatever port you configured) to perform administrative operations or produce or consume data. - -~~~~ -$ ./bin/kafka-topics.sh --create --topic foo --partitions 1 --replication-factor 1 --bootstrap-server localhost:9092 -Created topic foo. -~~~~ - -# Deployment - -## Controller Servers -In KRaft mode, only a small group of specially selected servers can act as controllers (unlike the ZooKeeper-based mode, where any server can become the -Controller). The specially selected controller servers will participate in the metadata quorum. Each controller server is either active, or a hot -standby for the current active controller server. - -You will typically select 3 or 5 servers for this role, depending on factors like cost and the number of concurrent failures your system should withstand -without availability impact. Just like with ZooKeeper, you must keep a majority of the controllers alive in order to maintain availability. So if you have 3 -controllers, you can tolerate 1 failure; with 5 controllers, you can tolerate 2 failures. - -## Process Roles -Each Kafka server now has a new configuration key called `process.roles` which can have the following values: - -* If `process.roles` is set to `broker`, the server acts as a broker in KRaft mode. -* If `process.roles` is set to `controller`, the server acts as a controller in KRaft mode. -* If `process.roles` is set to `broker,controller`, the server acts as both a broker and a controller in KRaft mode. -* If `process.roles` is not set at all then we are assumed to be in ZooKeeper mode. As mentioned earlier, you can't currently transition back and forth between ZooKeeper mode and KRaft mode without reformatting. - -Nodes that act as both brokers and controllers are referred to as "combined" nodes. Combined nodes are simpler to operate for simple use cases and allow you to avoid -some fixed memory overheads associated with JVMs. The key disadvantage is that the controller will be less isolated from the rest of the system. For example, if activity on the broker causes an out of -memory condition, the controller part of the server is not isolated from that OOM condition. - -## Quorum Voters -All nodes in the system must set the `controller.quorum.voters` configuration. This identifies the quorum controller servers that should be used. All the controllers must be enumerated. -This is similar to how, when using ZooKeeper, the `zookeeper.connect` configuration must contain all the ZooKeeper servers. Unlike with the ZooKeeper config, however, `controller.quorum.voters` -also has IDs for each node. The format is id1@host1:port1,id2@host2:port2, etc. - -So if you have 10 brokers and 3 controllers named controller1, controller2, controller3, you might have the following configuration on controller1: -``` -process.roles=controller -node.id=1 -listeners=CONTROLLER://controller1.example.com:9093 -controller.quorum.voters=1@controller1.example.com:9093,2@controller2.example.com:9093,3@controller3.example.com:9093 -``` - -Each broker and each controller must set `controller.quorum.voters`. Note that the node ID supplied in the `controller.quorum.voters` configuration must match that supplied to the server. -So on controller1, node.id must be set to 1, and so forth. Note that there is no requirement for controller IDs to start at 0 or 1. However, the easiest and least confusing way to allocate -node IDs is probably just to give each server a numeric ID, starting from 0. Also note that each node ID must be unique across all the nodes in a particular cluster; no two nodes can have the same node ID regardless of their `process.roles` values. - -Note that clients never need to configure `controller.quorum.voters`; only servers do. - -## Kafka Storage Tool -As described above in the QuickStart section, you must use the `kafka-storage.sh` tool to generate a cluster ID for your new cluster, and then run the format command on each node before starting the node. - -This is different from how Kafka has operated in the past. Previously, Kafka would format blank storage directories automatically, and also generate a new cluster UUID automatically. One reason for the change -is that auto-formatting can sometimes obscure an error condition. For example, under UNIX, if a data directory can't be mounted, it may show up as blank. In this case, auto-formatting would be the wrong thing to do. - -This is particularly important for the metadata log maintained by the controller servers. If two controllers out of three controllers were able to start with blank logs, a leader might be able to be elected with -nothing in the log, which would cause all metadata to be lost. - -# Missing Features -We don't support any kind of upgrade right now, either to or from KRaft mode. This is an important gap that we are working on. - -Finally, the following Kafka features have not yet been fully implemented: - -* Configuring SCRAM users via the administrative API -* Supporting JBOD configurations with multiple storage directories -* Modifying certain dynamic configurations on the standalone KRaft controller -* Support for some configurations, like enabling unclean leader election by default or dynamically changing broker endpoints -* Delegation tokens -* Upgrade from ZooKeeper mode - -We've tried to make it clear when a feature is not supported in the preview release, but you may encounter some rough edges. We will cover these feature gaps incrementally in the `trunk` branch. - -# Debugging -If you encounter an issue, you might want to take a look at the metadata log. - -## kafka-dump-log -One way to view the metadata log is with kafka-dump-log.sh tool, like so: - -~~~~ -$ ./bin/kafka-dump-log.sh --cluster-metadata-decoder --skip-record-metadata --files /tmp/kraft-combined-logs/__cluster_metadata-0/*.log -Dumping /tmp/kraft-combined-logs/__cluster_metadata-0/00000000000000000000.log -Starting offset: 0 -baseOffset: 0 lastOffset: 0 count: 1 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 1 isTransactional: false isControl: true position: 0 CreateTime: 1614382631640 size: 89 magic: 2 compresscodec: NONE crc: 1438115474 isvalid: true - -baseOffset: 1 lastOffset: 1 count: 1 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 1 isTransactional: false isControl: false position: 89 CreateTime: 1614382632329 size: 137 magic: 2 compresscodec: NONE crc: 1095855865 isvalid: true - payload: {"type":"REGISTER_BROKER_RECORD","version":0,"data":{"brokerId":1,"incarnationId":"P3UFsWoNR-erL9PK98YLsA","brokerEpoch":0,"endPoints":[{"name":"PLAINTEXT","host":"localhost","port":9092,"securityProtocol":0}],"features":[],"rack":null}} -baseOffset: 2 lastOffset: 2 count: 1 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 1 isTransactional: false isControl: false position: 226 CreateTime: 1614382632453 size: 83 magic: 2 compresscodec: NONE crc: 455187130 isvalid: true - payload: {"type":"UNFENCE_BROKER_RECORD","version":0,"data":{"id":1,"epoch":0}} -baseOffset: 3 lastOffset: 3 count: 1 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 1 isTransactional: false isControl: false position: 309 CreateTime: 1614382634484 size: 83 magic: 2 compresscodec: NONE crc: 4055692847 isvalid: true - payload: {"type":"FENCE_BROKER_RECORD","version":0,"data":{"id":1,"epoch":0}} -baseOffset: 4 lastOffset: 4 count: 1 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 2 isTransactional: false isControl: true position: 392 CreateTime: 1614382671857 size: 89 magic: 2 compresscodec: NONE crc: 1318571838 isvalid: true - -baseOffset: 5 lastOffset: 5 count: 1 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 2 isTransactional: false isControl: false position: 481 CreateTime: 1614382672440 size: 137 magic: 2 compresscodec: NONE crc: 841144615 isvalid: true - payload: {"type":"REGISTER_BROKER_RECORD","version":0,"data":{"brokerId":1,"incarnationId":"RXRJu7cnScKRZOnWQGs86g","brokerEpoch":4,"endPoints":[{"name":"PLAINTEXT","host":"localhost","port":9092,"securityProtocol":0}],"features":[],"rack":null}} -baseOffset: 6 lastOffset: 6 count: 1 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 2 isTransactional: false isControl: false position: 618 CreateTime: 1614382672544 size: 83 magic: 2 compresscodec: NONE crc: 4155905922 isvalid: true - payload: {"type":"UNFENCE_BROKER_RECORD","version":0,"data":{"id":1,"epoch":4}} -baseOffset: 7 lastOffset: 8 count: 2 baseSequence: -1 lastSequence: -1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 2 isTransactional: false isControl: false position: 701 CreateTime: 1614382712158 size: 159 magic: 2 compresscodec: NONE crc: 3726758683 isvalid: true - payload: {"type":"TOPIC_RECORD","version":0,"data":{"name":"foo","topicId":"5zoAlv-xEh9xRANKXt1Lbg"}} - payload: {"type":"PARTITION_RECORD","version":0,"data":{"partitionId":0,"topicId":"5zoAlv-xEh9xRANKXt1Lbg","replicas":[1],"isr":[1],"removingReplicas":null,"addingReplicas":null,"leader":1,"leaderEpoch":0,"partitionEpoch":0}} -~~~~ - -## The Metadata Shell -Another tool for examining the metadata logs is the Kafka metadata shell. Just like the ZooKeeper shell, this allows you to inspect the metadata of the cluster. - -~~~~ -$ ./bin/kafka-metadata-shell.sh --snapshot /tmp/kraft-combined-logs/__cluster_metadata-0/00000000000000000000.log ->> ls / -brokers local metadataQuorum topicIds topics ->> ls /topics -foo ->> cat /topics/foo/0/data -{ - "partitionId" : 0, - "topicId" : "5zoAlv-xEh9xRANKXt1Lbg", - "replicas" : [ 1 ], - "isr" : [ 1 ], - "removingReplicas" : null, - "addingReplicas" : null, - "leader" : 1, - "leaderEpoch" : 0, - "partitionEpoch" : 0 -} ->> exit -~~~~ diff --git a/config/log4j.properties b/config/log4j.properties index 4cbce9d104291..4dbdd83f83b74 100644 --- a/config/log4j.properties +++ b/config/log4j.properties @@ -76,6 +76,11 @@ log4j.additivity.kafka.request.logger=false log4j.logger.kafka.network.RequestChannel$=WARN, requestAppender log4j.additivity.kafka.network.RequestChannel$=false +# Change the line below to adjust KRaft mode controller logging +log4j.logger.org.apache.kafka.controller=INFO, controllerAppender +log4j.additivity.org.apache.kafka.controller=false + +# Change the line below to adjust ZK mode controller logging log4j.logger.kafka.controller=TRACE, controllerAppender log4j.additivity.kafka.controller=false diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java index d89f577688f35..407f5fd828b84 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java @@ -38,6 +38,7 @@ import org.apache.kafka.connect.header.Headers; import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; import org.apache.kafka.connect.runtime.errors.Stage; +import org.apache.kafka.connect.runtime.errors.ToleranceType; import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceTask; import org.apache.kafka.connect.source.SourceTaskContext; @@ -406,6 +407,10 @@ boolean sendRecords() { } log.trace("{} Failed record: {}", AbstractWorkerSourceTask.this, preTransformRecord); producerSendFailed(false, producerRecord, preTransformRecord, e); + if (retryWithToleranceOperator.getErrorToleranceType() == ToleranceType.ALL) { + counter.skipRecord(); + submittedRecord.ifPresent(SubmittedRecords.SubmittedRecord::ack); + } } else { counter.completeRecord(); log.trace("{} Wrote record successfully: topic {} partition {} offset {}", diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java index de78e592aa2a1..6443fbf630d2c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java @@ -217,6 +217,9 @@ protected void finalOffsetCommit(boolean failed) { if (failed) { log.debug("Skipping final offset commit as task has failed"); return; + } else if (isCancelled()) { + log.debug("Skipping final offset commit as task has been cancelled"); + return; } // It should be safe to commit here even if we were in the middle of retrying on RetriableExceptions in the @@ -275,18 +278,22 @@ private void commitTransaction() { }); } - // Commit the transaction - // Blocks until all outstanding records have been sent and ack'd - try { - producer.commitTransaction(); - } catch (Throwable t) { - log.error("{} Failed to commit producer transaction", ExactlyOnceWorkerSourceTask.this, t); - flushError.compareAndSet(null, t); + // Only commit the transaction if we were able to serialize the offsets. + // Otherwise, we may commit source records without committing their offsets + Throwable error = flushError.get(); + if (error == null) { + try { + // Commit the transaction + // Blocks until all outstanding records have been sent and ack'd + producer.commitTransaction(); + } catch (Throwable t) { + log.error("{} Failed to commit producer transaction", ExactlyOnceWorkerSourceTask.this, t); + flushError.compareAndSet(null, t); + } + transactionOpen = false; } - transactionOpen = false; - - Throwable error = flushError.get(); + error = flushError.get(); if (error != null) { recordCommitFailure(time.milliseconds() - started, null); offsetWriter.cancelFlush(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index 16e48d8f17edf..5bc67693d0a66 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -756,6 +756,19 @@ static Map exactlyOnceSourceTaskProducerConfigs(ConnectorTaskId ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, String clusterId) { Map result = baseProducerConfigs(id.connector(), "connector-producer-" + id, config, connConfig, connectorClass, connectorClientConfigOverridePolicy, clusterId); + // The base producer properties forcibly disable idempotence; remove it from those properties + // if not explicitly requested by the user + boolean connectorProducerIdempotenceConfigured = connConfig.originals().containsKey( + ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX + ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG + ); + if (!connectorProducerIdempotenceConfigured) { + boolean workerProducerIdempotenceConfigured = config.originals().containsKey( + "producer." + ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG + ); + if (!workerProducerIdempotenceConfigured) { + result.remove(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG); + } + } ConnectUtils.ensureProperty( result, ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true", "for connectors when exactly-once source support is enabled", diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index ded833da59914..3f6ff06e846b2 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -655,6 +655,7 @@ private void processConnectorConfigUpdates(Set connectorConfigUpdates) { // If we only have connector config updates, we can just bounce the updated connectors that are // currently assigned to this worker. Set localConnectors = assignment == null ? Collections.emptySet() : new HashSet<>(assignment.connectors()); + Collection> connectorsToStart = new ArrayList<>(); log.trace("Processing connector config updates; " + "currently-owned connectors are {}, and to-be-updated connectors are {}", localConnectors, @@ -671,13 +672,10 @@ private void processConnectorConfigUpdates(Set connectorConfigUpdates) { worker.stopAndAwaitConnector(connectorName); // The update may be a deletion, so verify we actually need to restart the connector if (remains) { - startConnector(connectorName, (error, result) -> { - if (error != null) { - log.error("Failed to start connector '" + connectorName + "'", error); - } - }); + connectorsToStart.add(getConnectorStartingCallable(connectorName)); } } + startAndStop(connectorsToStart); } private void processTargetStateChanges(Set connectorTargetStateChanges) { @@ -1732,9 +1730,7 @@ private boolean startTask(ConnectorTaskId taskId) { throw ConnectUtils.maybeWrap(cause, "Failed to perform round of zombie fencing"); } }, - () -> { - verifyTaskGenerationAndOwnership(taskId, taskGeneration); - } + () -> verifyTaskGenerationAndOwnership(taskId, taskGeneration) ); } else { return worker.startSourceTask( @@ -1941,8 +1937,8 @@ private void reconfigureConnector(final String connName, final Callback cb } } - // Currently unused, but will be invoked by exactly-once source tasks after they have successfully - // initialized their transactional producer + // Invoked by exactly-once worker source tasks after they have successfully initialized their transactional + // producer to ensure that it is still safe to bring up the task private void verifyTaskGenerationAndOwnership(ConnectorTaskId id, int initialTaskGen) { log.debug("Reading to end of config topic to ensure it is still safe to bring up source task {} with exactly-once support", id); if (!refreshConfigSnapshot(Long.MAX_VALUE)) { @@ -2404,13 +2400,16 @@ private boolean requestNotSignedProperly(InternalRequestSignature requestSignatu requestSignature.keyAlgorithm(), keySignatureVerificationAlgorithms )); - } else { - if (!requestSignature.isValid(sessionKey)) { - requestValidationError = new ConnectRestException( - Response.Status.FORBIDDEN, - "Internal request contained invalid signature." - ); - } + } else if (sessionKey == null) { + requestValidationError = new ConnectRestException( + Response.Status.SERVICE_UNAVAILABLE, + "This worker is still starting up and has not been able to read a session key from the config topic yet" + ); + } else if (!requestSignature.isValid(sessionKey)) { + requestValidationError = new ConnectRestException( + Response.Status.FORBIDDEN, + "Internal request contained invalid signature." + ); } if (requestValidationError != null) { callback.onCompletion(requestValidationError, null); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java index 1ab3543361211..ced67427a3f13 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java @@ -225,7 +225,7 @@ protected Map onLeaderElected(String leaderId, } @Override - protected boolean onJoinPrepare(int generation, String memberId) { + protected boolean onJoinPrepare(Timer timer, int generation, String memberId) { log.info("Rebalance started"); leaderState(null); final ExtendedAssignment localAssignmentSnapshot = assignmentSnapshot; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceTask.java index 9918752759c8b..ee49efed92085 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceTask.java @@ -18,6 +18,8 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; import org.apache.kafka.tools.ThroughputThrottler; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.connect.data.Schema; @@ -31,6 +33,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Stream; /** * A connector primarily intended for system tests. The connector simply generates as many tasks as requested. The @@ -48,6 +51,7 @@ public class VerifiableSourceTask extends SourceTask { public static final String ID_CONFIG = "id"; public static final String TOPIC_CONFIG = "topic"; public static final String THROUGHPUT_CONFIG = "throughput"; + public static final String COMPLETE_RECORD_DATA_CONFIG = "complete.record.data"; private static final String ID_FIELD = "id"; private static final String SEQNO_FIELD = "seqno"; @@ -61,6 +65,15 @@ public class VerifiableSourceTask extends SourceTask { private long startingSeqno; private long seqno; private ThroughputThrottler throttler; + private boolean completeRecordData; + + private static final Schema COMPLETE_VALUE_SCHEMA = SchemaBuilder.struct() + .field("name", Schema.STRING_SCHEMA) + .field("task", Schema.INT32_SCHEMA) + .field("topic", Schema.STRING_SCHEMA) + .field("time_ms", Schema.INT64_SCHEMA) + .field("seqno", Schema.INT64_SCHEMA) + .build(); @Override public String version() { @@ -87,6 +100,7 @@ public void start(Map props) { seqno = 0; startingSeqno = seqno; throttler = new ThroughputThrottler(throughput, System.currentTimeMillis()); + completeRecordData = "true".equalsIgnoreCase(props.get(COMPLETE_RECORD_DATA_CONFIG)); log.info("Started VerifiableSourceTask {}-{} producing to topic {} resuming from seqno {}", name, id, topic, startingSeqno); } @@ -114,7 +128,9 @@ public List poll() { System.out.println(dataJson); Map ccOffset = Collections.singletonMap(SEQNO_FIELD, seqno); - SourceRecord srcRecord = new SourceRecord(partition, ccOffset, topic, Schema.INT32_SCHEMA, id, Schema.INT64_SCHEMA, seqno); + Schema valueSchema = completeRecordData ? COMPLETE_VALUE_SCHEMA : Schema.INT64_SCHEMA; + Object value = completeRecordData ? completeValue(data) : seqno; + SourceRecord srcRecord = new SourceRecord(partition, ccOffset, topic, Schema.INT32_SCHEMA, id, valueSchema, value); List result = Collections.singletonList(srcRecord); seqno++; return result; @@ -141,6 +157,15 @@ public void commitRecord(SourceRecord record, RecordMetadata metadata) { @Override public void stop() { - throttler.wakeup(); + if (throttler != null) + throttler.wakeup(); + } + + private Object completeValue(Map data) { + Struct result = new Struct(COMPLETE_VALUE_SCHEMA); + Stream.of("name", "task", "topic", "time_ms", "seqno").forEach( + field -> result.put(field, data.get(field)) + ); + return result; } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java index 5da5be04a51f5..cde63b3f83392 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java @@ -166,6 +166,8 @@ public KafkaBasedLog(String topic, this.readLogEndOffsetCallbacks = new ArrayDeque<>(); this.time = time; this.initializer = initializer != null ? initializer : admin -> { }; + // Initialize the producer Optional here to prevent NPEs later on + this.producer = Optional.empty(); // If the consumer is configured with isolation.level = read_committed, then its end offsets method cannot be relied on // as it will not take records from currently-open transactions into account. We want to err on the side of caution in that 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 25a419ed8f8f3..bd9bceba064fd 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 @@ -76,6 +76,7 @@ import java.util.stream.LongStream; import static org.apache.kafka.clients.producer.ProducerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.CLIENT_ID_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.TRANSACTIONAL_ID_CONFIG; import static org.apache.kafka.connect.integration.MonitorableSourceConnector.CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG; @@ -406,12 +407,12 @@ public void testConnectorBoundary() throws Exception { Map consumerProps = new HashMap<>(); consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); // consume all records from the source topic or fail, to ensure that they were correctly produced - ConsumerRecords sourceRecords = connect.kafka() - .consume( - recordsProduced, - TimeUnit.MINUTES.toMillis(1), - consumerProps, - "test-topic"); + ConsumerRecords sourceRecords = connect.kafka().consumeAll( + CONSUME_RECORDS_TIMEOUT_MS, + Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), + null, + topic + ); assertTrue("Not enough records produced by source connector. Expected at least: " + recordsProduced + " + but got " + sourceRecords.count(), sourceRecords.count() >= recordsProduced); @@ -433,8 +434,7 @@ public void testConnectorBoundary() throws Exception { offsetsTopic ); - List actualOffsetSeqnos = new ArrayList<>(); - offsetRecords.forEach(record -> actualOffsetSeqnos.add(parseAndAssertOffsetForSingleTask(record))); + List actualOffsetSeqnos = parseAndAssertOffsetsForSingleTask(offsetRecords); assertEquals("Committed offsets should match connector-defined transaction boundaries", expectedOffsetSeqnos, actualOffsetSeqnos.subList(0, expectedOffsetSeqnos.size())); @@ -466,6 +466,7 @@ public void testConnectorBoundary() throws Exception { */ @Test public void testFencedLeaderRecovery() throws Exception { + connectBuilder.numWorkers(1); // Much slower offset commit interval; should never be triggered during this test workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG, "600000"); startConnect(); @@ -494,7 +495,10 @@ public void testFencedLeaderRecovery() throws Exception { assertEquals(404, connect.requestGet(connect.endpointForResource("connectors/nonexistent")).getStatus()); // fence out the leader of the cluster - Producer zombieLeader = transactionalProducer(DistributedConfig.transactionalProducerId(CLUSTER_GROUP_ID)); + Producer zombieLeader = transactionalProducer( + "simulated-zombie-leader", + DistributedConfig.transactionalProducerId(CLUSTER_GROUP_ID) + ); zombieLeader.initTransactions(); zombieLeader.close(); @@ -711,6 +715,20 @@ public void testTasksFailOnInabilityToFence() throws Exception { connect.assertions().assertConnectorAndExactlyNumTasksAreRunning(CONNECTOR_NAME, tasksMax, "Connector and task should have restarted successfully"); } + /** + * This test focuses extensively on the per-connector offsets feature. + *

+ * First, a connector is brought up whose producer is configured to write to a different Kafka cluster + * than the one the Connect cluster users for its internal topics, then the contents of the connector's + * dedicated offsets topic and the worker's internal offsets topic are inspected to ensure that offsets + * have been backed up from the dedicated topic to the global topic. + *

+ * Then, a "soft downgrade" is simulated: the Connect cluster is shut down and reconfigured to disable + * exactly-once support. The cluster is brought up again, the connector is allowed to produce some data, + * the connector is shut down, and this time, the records the connector has produced are inspected for + * accuracy. Because of the downgrade, exactly-once guarantees are lost, but we check to make sure that + * the task has maintained exactly-once delivery up to the last-committed record. + */ @Test public void testSeparateOffsetsTopic() throws Exception { final String globalOffsetsTopic = "connect-worker-offsets-topic"; @@ -756,7 +774,7 @@ public void testSeparateOffsetsTopic() throws Exception { // wait for the connector tasks to commit enough records connectorHandle.awaitCommits(TimeUnit.MINUTES.toMillis(1)); - // consume all records from the source topic or fail, to ensure that they were correctly produced + // consume at least the expected number of records from the source topic or fail, to ensure that they were correctly produced int recordNum = connectorTargetedCluster .consume( recordsProduced, @@ -767,28 +785,33 @@ public void testSeparateOffsetsTopic() throws Exception { assertTrue("Not enough records produced by source connector. Expected at least: " + recordsProduced + " + but got " + recordNum, recordNum >= recordsProduced); - // also consume from the connector's dedicated offsets topic; just need to read one offset record - ConsumerRecord offsetRecord = connectorTargetedCluster - .consume( - 1, + // also consume from the connector's dedicated offsets topic + ConsumerRecords offsetRecords = connectorTargetedCluster + .consumeAll( TimeUnit.MINUTES.toMillis(1), Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), + null, offsetsTopic - ).iterator().next(); - long seqno = parseAndAssertOffsetForSingleTask(offsetRecord); - assertEquals("Offset commits should occur on connector-defined poll boundaries, which happen every " + recordsProduced + " records", - 0, seqno % recordsProduced); + ); + List seqnos = parseAndAssertOffsetsForSingleTask(offsetRecords); + seqnos.forEach(seqno -> + assertEquals("Offset commits should occur on connector-defined poll boundaries, which happen every " + recordsProduced + " records", + 0, seqno % recordsProduced) + ); - // also consume from the cluster's global offsets topic; again, just need to read one offset record - offsetRecord = connect.kafka() - .consume( - 1, + // also consume from the cluster's global offsets topic + offsetRecords = connect.kafka() + .consumeAll( TimeUnit.MINUTES.toMillis(1), + null, + null, globalOffsetsTopic - ).iterator().next(); - seqno = parseAndAssertOffsetForSingleTask(offsetRecord); - assertEquals("Offset commits should occur on connector-defined poll boundaries, which happen every " + recordsProduced + " records", - 0, seqno % recordsProduced); + ); + seqnos = parseAndAssertOffsetsForSingleTask(offsetRecords); + seqnos.forEach(seqno -> + assertEquals("Offset commits should occur on connector-defined poll boundaries, which happen every " + recordsProduced + " records", + 0, seqno % recordsProduced) + ); // Shut down the whole cluster connect.workers().forEach(connect::removeWorker); @@ -826,15 +849,22 @@ public void testSeparateOffsetsTopic() throws Exception { assertConnectorStopped(connectorStop); // consume all records from the source topic or fail, to ensure that they were correctly produced - ConsumerRecords records = connectorTargetedCluster.consumeAll( + ConsumerRecords sourceRecords = connectorTargetedCluster.consumeAll( CONSUME_RECORDS_TIMEOUT_MS, Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), null, topic ); - assertTrue("Not enough records produced by source connector. Expected at least: " + recordsProduced + " + but got " + records.count(), - records.count() >= recordsProduced); - assertExactlyOnceSeqnos(records, numTasks); + assertTrue("Not enough records produced by source connector. Expected at least: " + recordsProduced + " + but got " + sourceRecords.count(), + sourceRecords.count() >= recordsProduced); + // also have to check which offsets have actually been committed, since we no longer have exactly-once guarantees + offsetRecords = connectorTargetedCluster.consumeAll( + CONSUME_RECORDS_TIMEOUT_MS, + Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), + null, + offsetsTopic + ); + assertAtLeastOnceSeqnos(sourceRecords, offsetRecords, numTasks); } } @@ -891,27 +921,10 @@ private ConfigInfo findConfigInfo(String property, ConfigInfos validationResult) .orElseThrow(() -> new AssertionError("Failed to find configuration validation result for property '" + property + "'")); } - @SuppressWarnings("unchecked") - private long parseAndAssertOffsetForSingleTask(ConsumerRecord offsetRecord) { - JsonConverter offsetsConverter = new JsonConverter(); - // The JSON converter behaves identically for keys and values. If that ever changes, we may need to update this test to use - // separate converter instances. - - offsetsConverter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), false); - Object keyObject = offsetsConverter.toConnectData("topic name is not used by converter", offsetRecord.key()).value(); - Object valueObject = offsetsConverter.toConnectData("topic name is not used by converter", offsetRecord.value()).value(); - - assertNotNull("Offset value should not be null", valueObject); - - assertEquals("Serialized source partition should match expected format", - Arrays.asList(CONNECTOR_NAME, MonitorableSourceConnector.sourcePartition(MonitorableSourceConnector.taskId(CONNECTOR_NAME, 0))), - keyObject); - - Map value = assertAndCast(valueObject, Map.class, "Value"); - - Object seqnoObject = value.get("saved"); - assertNotNull("Serialized source offset should contain 'seqno' field from MonitorableSourceConnector", seqnoObject); - return assertAndCast(seqnoObject, Long.class, "Seqno offset field"); + private List parseAndAssertOffsetsForSingleTask(ConsumerRecords offsetRecords) { + Map> parsedOffsets = parseOffsetForTasks(offsetRecords); + assertEquals("Expected records to only be produced from a single task", Collections.singleton(0), parsedOffsets.keySet()); + return parsedOffsets.get(0); } private List parseAndAssertValuesForSingleTask(ConsumerRecords sourceRecords) { @@ -922,6 +935,25 @@ private List parseAndAssertValuesForSingleTask(ConsumerRecords sourceRecords, int numTasks) { Map> parsedValues = parseValuesForTasks(sourceRecords); + assertSeqnos(parsedValues, numTasks); + } + + private void assertAtLeastOnceSeqnos(ConsumerRecords sourceRecords, ConsumerRecords offsetRecords, int numTasks) { + Map> parsedValues = parseValuesForTasks(sourceRecords); + Map lastCommittedValues = parseOffsetForTasks(offsetRecords) + .entrySet().stream().collect(Collectors.toMap( + Map.Entry::getKey, + e -> Collections.max(e.getValue()) + )); + parsedValues.replaceAll((task, values) -> { + Long committedValue = lastCommittedValues.get(task); + assertNotNull("No committed offset found for task " + task, committedValue); + return values.stream().filter(v -> v <= committedValue).collect(Collectors.toList()); + }); + assertSeqnos(parsedValues, numTasks); + } + + private void assertSeqnos(Map> parsedValues, int numTasks) { Set expectedKeys = IntStream.range(0, numTasks).boxed().collect(Collectors.toSet()); assertEquals("Expected records to be produced by each task", expectedKeys, parsedValues.keySet()); @@ -930,10 +962,19 @@ private void assertExactlyOnceSeqnos(ConsumerRecords sourceRecor // which makes in-order consumption impossible Set expectedSeqnos = LongStream.range(1, seqnos.size() + 1).boxed().collect(Collectors.toSet()); Set actualSeqnos = new HashSet<>(seqnos); - assertEquals( - "Seqnos for task " + taskId + " should start at 1 and increase strictly by 1 with each record", - expectedSeqnos, - actualSeqnos + + Set missingSeqnos = new HashSet<>(expectedSeqnos); + missingSeqnos.removeAll(actualSeqnos); + Set extraSeqnos = new HashSet<>(actualSeqnos); + extraSeqnos.removeAll(expectedSeqnos); + + // Try to provide the most friendly error message possible if this test fails + assertTrue( + "Seqnos for task " + taskId + " should start at 1 and increase strictly by 1 with each record, " + + "but the actual seqnos did not.\n" + + "Seqnos that should have been emitted but were not: " + missingSeqnos + "\n" + + "seqnos that should not have been emitted but were: " + extraSeqnos, + missingSeqnos.isEmpty() && extraSeqnos.isEmpty() ); }); } @@ -981,6 +1022,54 @@ private Map> parseValuesForTasks(ConsumerRecords> parseOffsetForTasks(ConsumerRecords offsetRecords) { + JsonConverter offsetsConverter = new JsonConverter(); + // The JSON converter behaves identically for keys and values. If that ever changes, we may need to update this test to use + // separate converter instances. + offsetsConverter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), false); + + Map> result = new HashMap<>(); + for (ConsumerRecord offsetRecord : offsetRecords) { + Object keyObject = offsetsConverter.toConnectData("topic name is not used by converter", offsetRecord.key()).value(); + Object valueObject = offsetsConverter.toConnectData("topic name is not used by converter", offsetRecord.value()).value(); + + assertNotNull("Offset key should not be null", keyObject); + assertNotNull("Offset value should not be null", valueObject); + + @SuppressWarnings("unchecked") + List key = assertAndCast(keyObject, List.class, "Key"); + assertEquals( + "Offset topic key should be a list containing two elements: the name of the connector, and the connector-provided source partition", + 2, + key.size() + ); + assertEquals(CONNECTOR_NAME, key.get(0)); + @SuppressWarnings("unchecked") + Map partition = assertAndCast(key.get(1), Map.class, "Key[1]"); + Object taskIdObject = partition.get("task.id"); + assertNotNull("Serialized source partition should contain 'task.id' field from MonitorableSourceConnector", taskIdObject); + String taskId = assertAndCast(taskIdObject, String.class, "task ID"); + assertTrue("task ID should match pattern '-", taskId.startsWith(CONNECTOR_NAME + "-")); + String taskIdRemainder = taskId.substring(CONNECTOR_NAME.length() + 1); + int taskNum; + try { + taskNum = Integer.parseInt(taskIdRemainder); + } catch (NumberFormatException e) { + throw new AssertionError("task ID should match pattern '-', where is an integer", e); + } + + @SuppressWarnings("unchecked") + Map value = assertAndCast(valueObject, Map.class, "Value"); + + Object seqnoObject = value.get("saved"); + assertNotNull("Serialized source offset should contain 'seqno' field from MonitorableSourceConnector", seqnoObject); + long seqno = assertAndCast(seqnoObject, Long.class, "Seqno offset field"); + + result.computeIfAbsent(taskNum, t -> new ArrayList<>()).add(seqno); + } + return result; + } + @SuppressWarnings("unchecked") private static T assertAndCast(Object o, Class klass, String objectDescription) { String className = o == null ? "null" : o.getClass().getName(); @@ -1030,9 +1119,10 @@ private void assertProducersAreFencedOnReconfiguration( // create a collection of producers that simulate the producers used for the existing tasks List> producers = IntStream.range(0, currentNumTasks) - .mapToObj(i -> Worker.taskTransactionalId(CLUSTER_GROUP_ID, CONNECTOR_NAME, i)) - .map(this::transactionalProducer) - .collect(Collectors.toList()); + .mapToObj(i -> transactionalProducer( + "simulated-task-producer-" + CONNECTOR_NAME + "-" + i, + Worker.taskTransactionalId(CLUSTER_GROUP_ID, CONNECTOR_NAME, i) + )).collect(Collectors.toList()); producers.forEach(KafkaProducer::initTransactions); @@ -1047,8 +1137,9 @@ private void assertProducersAreFencedOnReconfiguration( producers.forEach(producer -> assertTransactionalProducerIsFenced(producer, topic)); } - private KafkaProducer transactionalProducer(String transactionalId) { + private KafkaProducer transactionalProducer(String clientId, String transactionalId) { Map transactionalProducerProps = new HashMap<>(); + transactionalProducerProps.put(CLIENT_ID_CONFIG, clientId); transactionalProducerProps.put(ENABLE_IDEMPOTENCE_CONFIG, true); transactionalProducerProps.put(TRANSACTIONAL_ID_CONFIG, transactionalId); return connect.kafka().createProducer(transactionalProducerProps); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSourceConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSourceConnector.java index c2820315d6b82..33ba1588a7d04 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSourceConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSourceConnector.java @@ -172,7 +172,7 @@ public void start(Map props) { batchSize = Integer.parseInt(props.getOrDefault(MESSAGES_PER_POLL_CONFIG, "1")); taskHandle = RuntimeHandles.get().connectorHandle(connectorName).taskHandle(taskId); Map offset = Optional.ofNullable( - context.offsetStorageReader().offset(Collections.singletonMap("task.id", taskId))) + context.offsetStorageReader().offset(sourcePartition(taskId))) .orElse(Collections.emptyMap()); startingSeqno = Optional.ofNullable((Long) offset.get("saved")).orElse(0L); seqno = startingSeqno; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java index fdc253d04e234..981ad8c239d49 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java @@ -101,10 +101,10 @@ import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; @PowerMockIgnore({"javax.management.*", "org.apache.log4j.*"}) @@ -660,9 +660,9 @@ public void testIntervalBasedCommit() throws Exception { final CountDownLatch thirdPollLatch = new CountDownLatch(2); AtomicInteger flushes = new AtomicInteger(); - expectFlush(FlushOutcome.SUCCEED, flushes); - expectFlush(FlushOutcome.SUCCEED, flushes); - expectFlush(FlushOutcome.SUCCEED, flushes); + expectFlush(false, flushes); + expectFlush(false, flushes); + expectFlush(false, flushes); expectTopicCreation(TOPIC); @@ -717,20 +717,20 @@ public void testConnectorBasedCommit() throws Exception { AtomicInteger flushes = new AtomicInteger(); // First flush: triggered by TransactionContext::commitTransaction (batch) - expectFlush(FlushOutcome.SUCCEED, flushes); + expectFlush(false, flushes); // Second flush: triggered by TransactionContext::commitTransaction (record) - expectFlush(FlushOutcome.SUCCEED, flushes); + expectFlush(false, flushes); // Third flush: triggered by TransactionContext::abortTransaction (batch) expectCall(producer::abortTransaction); EasyMock.expect(offsetWriter.willFlush()).andReturn(true); - expectFlush(FlushOutcome.SUCCEED, flushes); + expectFlush(false, flushes); // Third flush: triggered by TransactionContext::abortTransaction (record) EasyMock.expect(offsetWriter.willFlush()).andReturn(true); expectCall(producer::abortTransaction); - expectFlush(FlushOutcome.SUCCEED, flushes); + expectFlush(false, flushes); expectTopicCreation(TOPIC); @@ -794,30 +794,57 @@ public void testConnectorBasedCommit() throws Exception { } @Test - public void testCommitFlushCallbackFailure() throws Exception { - testCommitFailure(FlushOutcome.FAIL_FLUSH_CALLBACK); + public void testCommitFlushSyncCallbackFailure() throws Exception { + Exception failure = new RecordTooLargeException(); + Capture> flushCallback = EasyMock.newCapture(); + EasyMock.expect(offsetWriter.doFlush(EasyMock.capture(flushCallback))).andAnswer(() -> { + flushCallback.getValue().onCompletion(failure, null); + return null; + }); + testCommitFailure(failure); + } + + @Test + public void testCommitFlushAsyncCallbackFailure() throws Exception { + Exception failure = new RecordTooLargeException(); + // doFlush delegates its callback to the producer, + // which delays completing the callback until commitTransaction + Capture> flushCallback = EasyMock.newCapture(); + EasyMock.expect(offsetWriter.doFlush(EasyMock.capture(flushCallback))).andReturn(null); + expectCall(producer::commitTransaction).andAnswer(() -> { + flushCallback.getValue().onCompletion(failure, null); + return null; + }); + testCommitFailure(failure); } @Test public void testCommitTransactionFailure() throws Exception { - testCommitFailure(FlushOutcome.FAIL_TRANSACTION_COMMIT); + Exception failure = new RecordTooLargeException(); + EasyMock.expect(offsetWriter.doFlush(EasyMock.anyObject())).andReturn(null); + expectCall(producer::commitTransaction).andThrow(failure); + testCommitFailure(failure); } - private void testCommitFailure(FlushOutcome causeOfFailure) throws Exception { + private void testCommitFailure(Throwable expectedFailure) throws Exception { createWorkerTask(); expectPreflight(); expectStartup(); expectPolls(); - expectFlush(causeOfFailure); + + EasyMock.expect(offsetWriter.willFlush()).andReturn(true).anyTimes(); + EasyMock.expect(offsetWriter.beginFlush()).andReturn(true); + expectCall(offsetWriter::cancelFlush); expectTopicCreation(TOPIC); expectCall(sourceTask::stop); // Unlike the standard WorkerSourceTask class, this one fails permanently when offset commits don't succeed final CountDownLatch taskFailure = new CountDownLatch(1); - expectCall(() -> statusListener.onFailure(EasyMock.eq(taskId), EasyMock.anyObject())) + Capture failure = EasyMock.newCapture(); + expectCall(() -> statusListener.onFailure(EasyMock.eq(taskId), EasyMock.capture(failure))) .andAnswer(() -> { taskFailure.countDown(); return null; @@ -833,6 +860,8 @@ private void testCommitFailure(FlushOutcome causeOfFailure) throws Exception { assertTrue(awaitLatch(taskFailure)); workerTask.stop(); assertTrue(workerTask.awaitStop(1000)); + assertNotNull(failure.getValue()); + assertEquals(expectedFailure, failure.getValue().getCause()); taskFuture.get(); assertPollMetrics(1); @@ -1156,7 +1185,7 @@ private enum FlushOutcome { FAIL_TRANSACTION_COMMIT } - private CountDownLatch expectFlush(FlushOutcome outcome, AtomicInteger flushCount) { + private CountDownLatch expectFlush(boolean anyTimes, AtomicInteger flushCount) { CountDownLatch result = new CountDownLatch(1); org.easymock.IExpectationSetters flushBegin = EasyMock .expect(offsetWriter.beginFlush()) @@ -1165,55 +1194,32 @@ private CountDownLatch expectFlush(FlushOutcome outcome, AtomicInteger flushCoun result.countDown(); return true; }); - if (FlushOutcome.SUCCEED_ANY_TIMES.equals(outcome)) { + if (anyTimes) { flushBegin.anyTimes(); } Capture> flushCallback = EasyMock.newCapture(); org.easymock.IExpectationSetters> offsetFlush = EasyMock.expect(offsetWriter.doFlush(EasyMock.capture(flushCallback))); - switch (outcome) { - case SUCCEED: - // The worker task doesn't actually use the returned future - offsetFlush.andReturn(null); - expectCall(producer::commitTransaction); - expectCall(() -> sourceTask.commitRecord(EasyMock.anyObject(), EasyMock.anyObject())); - expectCall(sourceTask::commit); - break; - case SUCCEED_ANY_TIMES: - // The worker task doesn't actually use the returned future - offsetFlush.andReturn(null).anyTimes(); - expectCall(producer::commitTransaction).anyTimes(); - expectCall(() -> sourceTask.commitRecord(EasyMock.anyObject(), EasyMock.anyObject())).anyTimes(); - expectCall(sourceTask::commit).anyTimes(); - break; - case FAIL_FLUSH_CALLBACK: - expectCall(producer::commitTransaction); - offsetFlush.andAnswer(() -> { - flushCallback.getValue().onCompletion(new RecordTooLargeException(), null); - return null; - }); - expectCall(offsetWriter::cancelFlush); - break; - case FAIL_TRANSACTION_COMMIT: - offsetFlush.andReturn(null); - expectCall(producer::commitTransaction) - .andThrow(new RecordTooLargeException()); - expectCall(offsetWriter::cancelFlush); - break; - default: - fail("Unexpected flush outcome: " + outcome); + if (anyTimes) { + // The worker task doesn't actually use the returned future + offsetFlush.andReturn(null).anyTimes(); + expectCall(producer::commitTransaction).anyTimes(); + expectCall(() -> sourceTask.commitRecord(EasyMock.anyObject(), EasyMock.anyObject())).anyTimes(); + expectCall(sourceTask::commit).anyTimes(); + } else { + // The worker task doesn't actually use the returned future + offsetFlush.andReturn(null); + expectCall(producer::commitTransaction); + expectCall(() -> sourceTask.commitRecord(EasyMock.anyObject(), EasyMock.anyObject())); + expectCall(sourceTask::commit); } return result; } - private CountDownLatch expectFlush(FlushOutcome outcome) { - return expectFlush(outcome, new AtomicInteger()); - } - private CountDownLatch expectAnyFlushes(AtomicInteger flushCount) { EasyMock.expect(offsetWriter.willFlush()).andReturn(true).anyTimes(); - return expectFlush(FlushOutcome.SUCCEED_ANY_TIMES, flushCount); + return expectFlush(true, flushCount); } private void assertTransactionMetrics(int minimumMaxSizeExpected) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index 5ce0e44f3ea90..2d2cd00cf5348 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -696,24 +696,39 @@ public void testSourceTaskIgnoresProducerException() throws Exception { createWorkerTaskWithErrorToleration(); expectTopicCreation(TOPIC); + //Use different offsets for each record so we can verify all were committed + final Map offset2 = Collections.singletonMap("key", 13); + // send two records // record 1 will succeed // record 2 will invoke the producer's failure callback, but ignore the exception via retryOperator // and no ConnectException will be thrown SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); - + SourceRecord record2 = new SourceRecord(PARTITION, offset2, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + expectOffsetFlush(true); expectSendRecordOnce(); expectSendRecordProducerCallbackFail(); sourceTask.commitRecord(EasyMock.anyObject(SourceRecord.class), EasyMock.isNull()); - EasyMock.expectLastCall(); + + //As of KAFKA-14079 all offsets should be committed, even for failed records (if ignored) + //Only the last offset will be passed to the method as everything up to that point is committed + //Before KAFKA-14079 offset 12 would have been passed and not 13 as it would have been unacked + offsetWriter.offset(PARTITION, offset2); + PowerMock.expectLastCall(); PowerMock.replayAll(); + //Send records and then commit offsets and verify both were committed and no exception Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); Whitebox.invokeMethod(workerTask, "sendRecords"); + Whitebox.invokeMethod(workerTask, "updateCommittableOffsets"); + workerTask.commitOffsets(); PowerMock.verifyAll(); + + //Double check to make sure all submitted records were cleared + assertEquals(0, ((SubmittedRecords) Whitebox.getInternalState(workerTask, + "submittedRecords")).records.size()); } @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 3249412259e36..d4f2aa9c68498 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 @@ -19,6 +19,7 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.common.errors.AuthorizationException; import org.apache.kafka.common.utils.MockTime; @@ -29,6 +30,7 @@ import org.apache.kafka.connect.errors.NotFoundException; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; import org.apache.kafka.connect.runtime.ConnectorConfig; +import org.apache.kafka.connect.runtime.ConnectorStatus; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.MockConnectMetrics; import org.apache.kafka.connect.runtime.RestartPlan; @@ -72,6 +74,7 @@ import org.easymock.EasyMock; import org.junit.After; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.api.easymock.PowerMock; @@ -105,6 +108,8 @@ import static java.util.Collections.singletonList; import static javax.ws.rs.core.Response.Status.FORBIDDEN; +import static javax.ws.rs.core.Response.Status.SERVICE_UNAVAILABLE; +import static org.apache.kafka.connect.runtime.AbstractStatus.State.FAILED; import static org.apache.kafka.connect.runtime.SourceConnectorConfig.ExactlyOnceSupportLevel.REQUIRED; import static org.apache.kafka.connect.runtime.distributed.ConnectProtocol.CONNECT_PROTOCOL_V0; import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG; @@ -1904,6 +1909,85 @@ public void testConnectorConfigUpdate() throws Exception { PowerMock.verifyAll(); } + @Test + @Ignore( + "When enabled, this test causes others in this test suite to hang. " + + "Trunk has been updated to handle this issue, but the changes " + + "required to accomplish that have not been backported. " + + "See https://github.com/apache/kafka/pull/12295, " + + "https://github.com/apache/kafka/pull/12828, and " + + "https://github.com/apache/kafka/pull/12817" + ) + public void testConnectorConfigUpdateFailedTransformation() throws Exception { + // Connector config can be applied without any rebalance + + EasyMock.expect(member.memberId()).andStubReturn("member"); + EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); + + WorkerConfigTransformer configTransformer = EasyMock.mock(WorkerConfigTransformer.class); + ClusterConfigState snapshotWithTransform = new ClusterConfigState(1, null, Collections.singletonMap(CONN1, 1), + Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.STARTED), + TASK_CONFIGS_MAP, Collections.emptyMap(), Collections.emptyMap(), Collections.emptySet(), Collections.emptySet(), configTransformer); + + // join + expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); + EasyMock.expect(configTransformer.transform(EasyMock.eq(CONN1), EasyMock.anyObject())) + .andReturn(CONN1_CONFIG).times(2); // once for the connector, once for the single task + expectConfigRefreshAndSnapshot(snapshotWithTransform); + EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + Capture> onStart = newCapture(); + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); + PowerMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; + }); + member.wakeup(); + PowerMock.expectLastCall(); + EasyMock.expect(worker.getPlugins()).andReturn(plugins); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); + EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + // apply config + member.wakeup(); + PowerMock.expectLastCall(); + member.ensureActive(); + PowerMock.expectLastCall(); + EasyMock.expect(configBackingStore.snapshot()).andReturn(snapshotWithTransform); // for this test, it doesn't matter if we use the same config snapshot + EasyMock.expect(configTransformer.transform(EasyMock.eq(CONN1), EasyMock.anyObject())) + .andThrow(new ConfigException("Simulated exception thrown during config transformation")); + worker.stopAndAwaitConnector(CONN1); + PowerMock.expectLastCall(); + Capture failedStatus = newCapture(); + statusBackingStore.putSafe(capture(failedStatus)); + PowerMock.expectLastCall(); + EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + // These will occur just before/during the third tick + member.ensureActive(); + PowerMock.expectLastCall(); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + EasyMock.replay(configTransformer); + PowerMock.replayAll(); + + herder.tick(); // join + configUpdateListener.onConnectorConfigUpdate(CONN1); // read updated config + herder.tick(); // apply config + herder.tick(); + + PowerMock.verifyAll(); + + assertEquals(CONN1, failedStatus.getValue().id()); + assertEquals(FAILED, failedStatus.getValue().state()); + } + @Test public void testConnectorPaused() throws Exception { // ensure that target state changes are propagated to the worker @@ -2773,7 +2857,15 @@ public void testPutTaskConfigsInvalidSignature() { EasyMock.expect(signature.keyAlgorithm()).andReturn("HmacSHA256").anyTimes(); EasyMock.expect(signature.isValid(EasyMock.anyObject())).andReturn(false).anyTimes(); - PowerMock.replayAll(taskConfigCb, signature); + SessionKey sessionKey = EasyMock.mock(SessionKey.class); + SecretKey secretKey = EasyMock.niceMock(SecretKey.class); + EasyMock.expect(sessionKey.key()).andReturn(secretKey); + EasyMock.expect(sessionKey.creationTimestamp()).andReturn(time.milliseconds()); + + PowerMock.replayAll(taskConfigCb, signature, sessionKey, secretKey); + + // Read a new session key from the config topic + configUpdateListener.onSessionKeyUpdate(sessionKey); herder.putTaskConfigs(CONN1, TASK_CONFIGS, taskConfigCb, signature); @@ -2782,6 +2874,28 @@ public void testPutTaskConfigsInvalidSignature() { assertEquals(FORBIDDEN.getStatusCode(), ((ConnectRestException) errorCapture.getValue()).statusCode()); } + @Test + public void putTaskConfigsWorkerStillStarting() { + Callback taskConfigCb = EasyMock.mock(Callback.class); + Capture errorCapture = Capture.newInstance(); + taskConfigCb.onCompletion(capture(errorCapture), EasyMock.eq(null)); + EasyMock.expectLastCall().once(); + + EasyMock.expect(member.currentProtocolVersion()).andReturn(CONNECT_PROTOCOL_V2).anyTimes(); + + InternalRequestSignature signature = EasyMock.mock(InternalRequestSignature.class); + EasyMock.expect(signature.keyAlgorithm()).andReturn("HmacSHA256").anyTimes(); + EasyMock.expect(signature.isValid(EasyMock.anyObject())).andReturn(true).anyTimes(); + + PowerMock.replayAll(taskConfigCb, signature); + + herder.putTaskConfigs(CONN1, TASK_CONFIGS, taskConfigCb, signature); + + PowerMock.verifyAll(); + assertTrue(errorCapture.getValue() instanceof ConnectRestException); + assertEquals(SERVICE_UNAVAILABLE.getStatusCode(), ((ConnectRestException) errorCapture.getValue()).statusCode()); + } + @Test public void testPutTaskConfigsValidRequiredSignature() { Callback taskConfigCb = EasyMock.mock(Callback.class); @@ -2794,7 +2908,15 @@ public void testPutTaskConfigsValidRequiredSignature() { EasyMock.expect(signature.keyAlgorithm()).andReturn("HmacSHA256").anyTimes(); EasyMock.expect(signature.isValid(EasyMock.anyObject())).andReturn(true).anyTimes(); - PowerMock.replayAll(taskConfigCb, signature); + SessionKey sessionKey = EasyMock.mock(SessionKey.class); + SecretKey secretKey = EasyMock.niceMock(SecretKey.class); + EasyMock.expect(sessionKey.key()).andReturn(secretKey); + EasyMock.expect(sessionKey.creationTimestamp()).andReturn(time.milliseconds()); + + PowerMock.replayAll(taskConfigCb, signature, sessionKey, secretKey); + + // Read a new session key from the config topic + configUpdateListener.onSessionKeyUpdate(sessionKey); herder.putTaskConfigs(CONN1, TASK_CONFIGS, taskConfigCb, signature); @@ -2893,7 +3015,15 @@ public void testFenceZombiesInvalidSignature() { EasyMock.expect(signature.keyAlgorithm()).andReturn("HmacSHA256").anyTimes(); EasyMock.expect(signature.isValid(EasyMock.anyObject())).andReturn(false).anyTimes(); - PowerMock.replayAll(taskConfigCb, signature); + SessionKey sessionKey = EasyMock.mock(SessionKey.class); + SecretKey secretKey = EasyMock.niceMock(SecretKey.class); + EasyMock.expect(sessionKey.key()).andReturn(secretKey); + EasyMock.expect(sessionKey.creationTimestamp()).andReturn(time.milliseconds()); + + PowerMock.replayAll(taskConfigCb, signature, sessionKey, secretKey); + + // Read a new session key from the config topic + configUpdateListener.onSessionKeyUpdate(sessionKey); herder.fenceZombieSourceTasks(CONN1, taskConfigCb, signature); diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 4676bfd1013a7..9a42f9b874dc3 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -211,7 +211,7 @@ object ConfigCommand extends Logging { encoderConfigs.get(KafkaConfig.PasswordEncoderSecretProp) val encoderSecret = encoderConfigs.getOrElse(KafkaConfig.PasswordEncoderSecretProp, throw new IllegalArgumentException("Password encoder secret not specified")) - new PasswordEncoder(new Password(encoderSecret), + PasswordEncoder.encrypting(new Password(encoderSecret), None, encoderConfigs.get(KafkaConfig.PasswordEncoderCipherAlgorithmProp).getOrElse(Defaults.PasswordEncoderCipherAlgorithm), encoderConfigs.get(KafkaConfig.PasswordEncoderKeyLengthProp).map(_.toInt).getOrElse(Defaults.PasswordEncoderKeyLength), diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index 47c1d173b306b..8cc056f617e0f 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -18,7 +18,7 @@ package kafka.admin import java.time.{Duration, Instant} -import java.util.Properties +import java.util.{Collections, Properties} import com.fasterxml.jackson.dataformat.csv.CsvMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import kafka.utils._ @@ -753,9 +753,9 @@ object ConsumerGroupCommand extends Logging { private def getCommittedOffsets(groupId: String): Map[TopicPartition, OffsetAndMetadata] = { adminClient.listConsumerGroupOffsets( - groupId, - withTimeoutMs(new ListConsumerGroupOffsetsOptions) - ).partitionsToOffsetAndMetadata.get.asScala + Collections.singletonMap(groupId, new ListConsumerGroupOffsetsSpec), + withTimeoutMs(new ListConsumerGroupOffsetsOptions()) + ).partitionsToOffsetAndMetadata(groupId).get().asScala } type GroupMetadata = immutable.Map[String, immutable.Map[TopicPartition, OffsetAndMetadata]] @@ -988,8 +988,8 @@ object ConsumerGroupCommand extends Logging { 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-period, --to-earliest, " + - "--to-latest, --shift-by, --from-file, --to-current." + 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'." val DryRunDoc = "Only show results without executing changes on Consumer Groups. Supported operations: reset-offsets." val ExecuteDoc = "Execute operation. Supported operations: reset-offsets." diff --git a/core/src/main/scala/kafka/admin/FeatureCommand.scala b/core/src/main/scala/kafka/admin/FeatureCommand.scala index c5c62648f4ed9..3ed99022ed196 100644 --- a/core/src/main/scala/kafka/admin/FeatureCommand.scala +++ b/core/src/main/scala/kafka/admin/FeatureCommand.scala @@ -20,28 +20,33 @@ package kafka.admin import kafka.tools.TerseFailure import kafka.utils.Exit import net.sourceforge.argparse4j.ArgumentParsers -import net.sourceforge.argparse4j.impl.Arguments.{append, fileType, storeTrue} -import net.sourceforge.argparse4j.inf.{Namespace, Subparsers} +import net.sourceforge.argparse4j.impl.Arguments.{append, fileType, store, storeTrue} +import net.sourceforge.argparse4j.inf.{ArgumentParserException, Namespace, Subparsers} +import net.sourceforge.argparse4j.internal.HelpScreenException import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.clients.admin.FeatureUpdate.UpgradeType -import org.apache.kafka.clients.admin.{Admin, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult} +import org.apache.kafka.clients.admin.{Admin, FeatureUpdate, UpdateFeaturesOptions} import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.common.MetadataVersion -import java.io.File +import java.io.{File, PrintStream} import java.util.Properties -import scala.collection.Seq import scala.concurrent.ExecutionException import scala.jdk.CollectionConverters._ +import scala.compat.java8.OptionConverters._ object FeatureCommand { - def main(args: Array[String]): Unit = { - val res = mainNoExit(args) + val res = mainNoExit(args, System.out) Exit.exit(res) } - // This is used for integration tests in order to avoid killing the test with Exit.exit - def mainNoExit(args: Array[String]): Int = { + // This is used for integration tests in order to avoid killing the test with Exit.exit, + // and in order to capture the command output. + def mainNoExit( + args: Array[String], + out: PrintStream + ): Int = { val parser = ArgumentParsers.newArgumentParser("kafka-features") .defaultHelp(true) .description("This tool manages feature flags in Kafka.") @@ -59,7 +64,7 @@ object FeatureCommand { addDisableParser(subparsers) try { - val namespace = parser.parseArgsOrFail(args) + val namespace = parser.parseArgs(args) val command = namespace.getString("command") val commandConfig = namespace.get[File]("command_config") @@ -73,16 +78,23 @@ object FeatureCommand { } props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, namespace.getString("bootstrap_server")) val admin = Admin.create(props) - - command match { - case "describe" => handleDescribe(namespace, admin) - case "upgrade" => handleUpgrade(namespace, admin) - case "downgrade" => handleDowngrade(namespace, admin) - case "disable" => handleDisable(namespace, admin) + try { + command match { + case "describe" => handleDescribe(out, admin) + case "upgrade" => handleUpgrade(out, namespace, admin) + case "downgrade" => handleDowngrade(out, namespace, admin) + case "disable" => handleDisable(out, namespace, admin) + } + } finally { + admin.close() } - admin.close() 0 } catch { + case _: HelpScreenException => + 0 + case e: ArgumentParserException => + System.err.println(s"Command line error: ${e.getMessage}. Type --help for help.") + 1 case e: TerseFailure => System.err.println(e.getMessage) 1 @@ -90,175 +102,185 @@ object FeatureCommand { } def addDescribeParser(subparsers: Subparsers): Unit = { - val describeParser = subparsers.addParser("describe") - .help("Describe one or more feature flags.") - - val featureArgs = describeParser.addArgumentGroup("Specific Features") - featureArgs.addArgument("--feature") - .action(append()) - .help("A specific feature to describe. This option may be repeated for describing multiple feature flags.") - - val releaseArgs = describeParser.addArgumentGroup("All Features for release") - releaseArgs.addArgument("--release") + subparsers.addParser("describe") + .help("Describes the current active feature flags.") } def addUpgradeParser(subparsers: Subparsers): Unit = { val upgradeParser = subparsers.addParser("upgrade") .help("Upgrade one or more feature flags.") - - val featureArgs = upgradeParser.addArgumentGroup("Upgrade specific features") - featureArgs.addArgument("--feature") + upgradeParser.addArgument("--metadata") + .help("The level to which we should upgrade the metadata. For example, 3.3-IV3.") + .action(store()) + upgradeParser.addArgument("--feature") + .help("A feature upgrade we should perform, in feature=level format. For example: `metadata.version=5`.") .action(append()) - .help("A feature flag to upgrade. This option may be repeated for upgrading multiple feature flags.") - featureArgs.addArgument("--version") - .`type`(classOf[Short]) - .help("The version to upgrade to.") - .action(append()) - - val releaseArgs = upgradeParser.addArgumentGroup("Upgrade to feature level defined for a given release") - releaseArgs.addArgument("--release") - upgradeParser.addArgument("--dry-run") - .help("Perform a dry-run of this upgrade operation.") + .help("Validate this upgrade, but do not perform it.") .action(storeTrue()) } def addDowngradeParser(subparsers: Subparsers): Unit = { val downgradeParser = subparsers.addParser("downgrade") - .help("Upgrade one or more feature flags.") - + .help("Downgrade one or more feature flags.") + downgradeParser.addArgument("--metadata") + .help("The level to which we should downgrade the metadata. For example, 3.3-IV0.") + .action(store()) downgradeParser.addArgument("--feature") - .help("A feature flag to downgrade. This option may be repeated for downgrade multiple feature flags.") - .required(true) - .action(append()) - downgradeParser.addArgument("--version") - .`type`(classOf[Short]) - .help("The version to downgrade to.") - .required(true) + .help("A feature downgrade we should perform, in feature=level format. For example: `metadata.version=5`.") .action(append()) downgradeParser.addArgument("--unsafe") - .help("Perform this downgrade even if it considered unsafe. Refer to specific feature flag documentation for details.") + .help("Perform this downgrade even if it may irreversibly destroy metadata.") .action(storeTrue()) downgradeParser.addArgument("--dry-run") - .help("Perform a dry-run of this downgrade operation.") + .help("Validate this downgrade, but do not perform it.") .action(storeTrue()) } def addDisableParser(subparsers: Subparsers): Unit = { val disableParser = subparsers.addParser("disable") .help("Disable one or more feature flags. This is the same as downgrading the version to zero.") - disableParser.addArgument("--feature") - .help("A feature flag to disable. This option may be repeated for disable multiple feature flags.") - .required(true) + .help("A feature flag to disable.") .action(append()) disableParser.addArgument("--unsafe") - .help("Disable the feature flag(s) even if it considered unsafe. Refer to specific feature flag documentation for details.") + .help("Disable this feature flag even if it may irreversibly destroy metadata.") .action(storeTrue()) disableParser.addArgument("--dry-run") .help("Perform a dry-run of this disable operation.") .action(storeTrue()) } - def handleDescribe(namespace: Namespace, admin: Admin): Unit = { - val featureFilter = parseFeaturesOrRelease(namespace) match { - case Neither() => (_: String) => true - case Features(featureNames) => (feature: String) => featureNames.contains(feature) - case Release(release) => - // Special case, print the versions associated with the given release - printReleaseFeatures(release) - return - case Both() => throw new TerseFailure("Only one of --release or --feature may be specified with describe sub-command.") + def levelToString( + feature: String, + level: Short + ): String = { + if (feature.equals(MetadataVersion.FEATURE_NAME)) { + try { + MetadataVersion.fromFeatureLevel(level).version() + } catch { + case e: Throwable => s"UNKNOWN [${level}]" + } + } else { + level.toString } + } + def handleDescribe( + out: PrintStream, + admin: Admin + ): Unit = { val featureMetadata = admin.describeFeatures().featureMetadata().get() - val featureEpoch = featureMetadata.finalizedFeaturesEpoch() - val epochString = if (featureEpoch.isPresent) { - s"Epoch: ${featureEpoch.get}" - } else { - "Epoch: -" - } - val finalized = featureMetadata.finalizedFeatures().asScala - featureMetadata.supportedFeatures().asScala.foreach { - case (feature, range) => - if (featureFilter.apply(feature)) { - if (finalized.contains(feature)) { - println(s"Feature: $feature\tSupportedMinVersion: ${range.minVersion()}\t" + - s"SupportedMaxVersion: ${range.maxVersion()}\tFinalizedVersionLevel: ${finalized(feature).maxVersionLevel()}\t$epochString") - } else { - println(s"Feature: $feature\tSupportedMinVersion: ${range.minVersion()}\t" + - s"SupportedMaxVersion: ${range.maxVersion()}\tFinalizedVersionLevel: -\t$epochString") - } + val featureList = new java.util.TreeSet[String](featureMetadata.supportedFeatures().keySet()) + featureList.forEach { + case feature => + val finalizedLevel = featureMetadata.finalizedFeatures().asScala.get(feature) match { + case None => 0.toShort + case Some(v) => v.maxVersionLevel() } + val range = featureMetadata.supportedFeatures().get(feature) + out.printf("Feature: %s\tSupportedMinVersion: %s\tSupportedMaxVersion: %s\tFinalizedVersionLevel: %s\tEpoch: %s%n", + feature, + levelToString(feature, range.minVersion()), + levelToString(feature, range.maxVersion()), + levelToString(feature, finalizedLevel), + featureMetadata.finalizedFeaturesEpoch().asScala.flatMap(e => Some(e.toString)).getOrElse("-")) } } - def printReleaseFeatures(release: String): Unit = { - println(s"Default feature versions for release $release:") + def metadataVersionsToString(first: MetadataVersion, last: MetadataVersion): String = { + MetadataVersion.VERSIONS.toList.asJava. + subList(first.ordinal(), last.ordinal() + 1). + asScala.mkString(", ") } - def handleUpgrade(namespace: Namespace, admin: Admin): Unit = { - val featuresToUpgrade = parseFeaturesOrRelease(namespace) match { - case Features(featureNames) => parseVersions(featureNames, namespace) - case Release(release) => featuresForRelease(release) - case Neither() => throw new TerseFailure("Must specify either --release or at least one --feature and --version with upgrade sub-command.") - case Both() => throw new TerseFailure("Cannot specify both --release and --feature with upgrade sub-command.") - } - - val dryRun = namespace.getBoolean("dry_run") - val updateResult = admin.updateFeatures(featuresToUpgrade.map { case (feature, version) => - feature -> new FeatureUpdate(version, UpgradeType.UPGRADE) - }.asJava, new UpdateFeaturesOptions().validateOnly(dryRun)) - handleUpdateFeaturesResponse(updateResult, featuresToUpgrade, dryRun, "upgrade") + def handleUpgrade(out: PrintStream, namespace: Namespace, admin: Admin): Unit = { + handleUpgradeOrDowngrade("upgrade", out, namespace, admin, UpgradeType.UPGRADE) } - def handleDowngrade(namespace: Namespace, admin: Admin): Unit = { - val featuresToDowngrade = parseFeaturesOrRelease(namespace) match { - case Features(featureNames) => parseVersions(featureNames, namespace) - case Neither() => throw new TerseFailure("Must specify at least one --feature and --version with downgrade sub-command.") - case _ => throw new IllegalStateException() - } - - val dryRun = namespace.getBoolean("dry_run") + def downgradeType(namespace: Namespace): UpgradeType = { val unsafe = namespace.getBoolean("unsafe") - val updateResult = admin.updateFeatures(featuresToDowngrade.map { case (feature, version) => - if (unsafe) { - feature -> new FeatureUpdate(version, UpgradeType.UNSAFE_DOWNGRADE) - } else { - feature -> new FeatureUpdate(version, UpgradeType.SAFE_DOWNGRADE) - } - }.asJava, new UpdateFeaturesOptions().validateOnly(dryRun)) + if (unsafe == null || !unsafe) { + UpgradeType.SAFE_DOWNGRADE + } else { + UpgradeType.UNSAFE_DOWNGRADE + } + } - handleUpdateFeaturesResponse(updateResult, featuresToDowngrade, dryRun, "downgrade") + def handleDowngrade(out: PrintStream, namespace: Namespace, admin: Admin): Unit = { + handleUpgradeOrDowngrade("downgrade", out, namespace, admin, downgradeType(namespace)) } - def handleDisable(namespace: Namespace, admin: Admin): Unit = { - val featuresToDisable = parseFeaturesOrRelease(namespace) match { - case Features(featureNames) => featureNames - case Neither() => throw new TerseFailure("Must specify at least one --feature and --version with downgrade sub-command.") - case _ => throw new IllegalStateException() + def parseNameAndLevel(input: String): (String, Short) = { + val equalsIndex = input.indexOf("=") + if (equalsIndex < 0) { + throw new TerseFailure(s"Can't parse feature=level string ${input}: equals sign not found.") + } + val name = input.substring(0, equalsIndex).trim + val levelString = input.substring(equalsIndex + 1).trim + val level = try { + levelString.toShort + } catch { + case e: Throwable => throw new TerseFailure(s"Can't parse feature=level string ${input}: " + + s"unable to parse ${levelString} as a short.") } + (name, level) + } - val dryRun = namespace.getBoolean("dry_run") - val unsafe = namespace.getBoolean("unsafe") - val updateResult = admin.updateFeatures(featuresToDisable.map { feature => - if (unsafe) { - feature -> new FeatureUpdate(0.toShort, UpgradeType.UNSAFE_DOWNGRADE) - } else { - feature -> new FeatureUpdate(0.toShort, UpgradeType.SAFE_DOWNGRADE) + def handleUpgradeOrDowngrade( + op: String, + out: PrintStream, + namespace: Namespace, + admin: Admin, + upgradeType: UpgradeType + ): Unit = { + val updates = new java.util.HashMap[String, FeatureUpdate]() + Option(namespace.getString("metadata")).foreach(metadata => { + val version = try { + MetadataVersion.fromVersionString(metadata) + } catch { + case e: Throwable => throw new TerseFailure("Unsupported metadata version " + metadata + + ". Supported metadata versions are " + metadataVersionsToString( + MetadataVersion.MINIMUM_BOOTSTRAP_VERSION, MetadataVersion.latest())) } - }.toMap.asJava, new UpdateFeaturesOptions().validateOnly(dryRun)) + updates.put(MetadataVersion.FEATURE_NAME, new FeatureUpdate(version.featureLevel(), upgradeType)) + }) + Option(namespace.getList[String]("feature")).foreach(features => { + features.forEach(feature => { + val (name, level) = parseNameAndLevel(feature) + if (updates.put(name, new FeatureUpdate(level, upgradeType)) != null) { + throw new TerseFailure(s"Feature ${name} was specified more than once.") + } + }) + }) + update(op, out, admin, updates, namespace.getBoolean("dry-run")) + } - handleUpdateFeaturesResponse(updateResult, featuresToDisable.map { - feature => feature -> 0.toShort - }.toMap, dryRun, "disable") + def handleDisable(out: PrintStream, namespace: Namespace, admin: Admin): Unit = { + val upgradeType = downgradeType(namespace) + val updates = new java.util.HashMap[String, FeatureUpdate]() + Option(namespace.getList[String]("feature")).foreach(features => { + features.forEach(name => + if (updates.put(name, new FeatureUpdate(0.toShort, upgradeType)) != null) { + throw new TerseFailure(s"Feature ${name} was specified more than once.") + }) + } + ) + update("disable", out, admin, updates, namespace.getBoolean("dry-run")) } - def handleUpdateFeaturesResponse(updateResult: UpdateFeaturesResult, - updatedFeatures: Map[String, Short], - dryRun: Boolean, - op: String): Unit = { - val errors = updateResult.values().asScala.map { case (feature, future) => + def update( + op: String, + out: PrintStream, + admin: Admin, + updates: java.util.HashMap[String, FeatureUpdate], + dryRun: Boolean + ): Unit = { + if (updates.isEmpty) { + throw new TerseFailure(s"You must specify at least one feature to ${op}") + } + val result = admin.updateFeatures(updates, new UpdateFeaturesOptions().validateOnly(dryRun)) + val errors = result.values().asScala.map { case (feature, future) => try { future.get() feature -> None @@ -267,67 +289,39 @@ object FeatureCommand { case t: Throwable => feature -> Some(t) } } - - errors.foreach { case (feature, maybeThrowable) => + var numFailures = 0 + errors.keySet.toList.sorted.foreach { feature => + val maybeThrowable = errors(feature) + val level = updates.get(feature).maxVersionLevel() if (maybeThrowable.isDefined) { - if (dryRun) { - System.out.println(s"Can not $op feature '$feature' to ${updatedFeatures(feature)}. ${maybeThrowable.get.getMessage}") + val helper = if (dryRun) { + "Can not" + } else { + "Could not" + } + val suffix = if (op.equals("disable")) { + s"disable ${feature}" } else { - System.out.println(s"Could not $op feature '$feature' to ${updatedFeatures(feature)}. ${maybeThrowable.get.getMessage}") + s"${op} ${feature} to ${level}" } + out.println(s"${helper} ${suffix}. ${maybeThrowable.get.getMessage}") + numFailures = numFailures + 1 } else { - if (dryRun) { - System.out.println(s"Feature '$feature' can be ${op}d to ${updatedFeatures(feature)}.") + val verb = if (dryRun) { + "can be" + } else { + "was" + } + val obj = if (op.equals("disable")) { + "disabled." } else { - System.out.println(s"Feature '$feature' was ${op}d to ${updatedFeatures(feature)}.") + s"${op}d to ${level}." } + out.println(s"${feature} ${verb} ${obj}") } } - } - - sealed trait ReleaseOrFeatures { } - case class Neither() extends ReleaseOrFeatures - case class Release(release: String) extends ReleaseOrFeatures - case class Features(featureNames: Seq[String]) extends ReleaseOrFeatures - case class Both() extends ReleaseOrFeatures - - def parseFeaturesOrRelease(namespace: Namespace): ReleaseOrFeatures = { - val release = namespace.getString("release") - val features = namespace.getList[String]("feature").asScala - - if (release != null && features != null) { - Both() - } else if (release == null && features == null) { - Neither() - } else if (release != null) { - Release(release) - } else { - Features(features) - } - } - - def parseVersions(features: Seq[String], namespace: Namespace): Map[String, Short] = { - val versions = namespace.getList[Short]("version").asScala - if (versions == null) { - throw new TerseFailure("Must specify --version when using --feature argument(s).") - } - if (versions.size != features.size) { - if (versions.size > features.size) { - throw new TerseFailure("Too many --version arguments given. For each --feature argument there should be one --version argument.") - } else { - throw new TerseFailure("Too many --feature arguments given. For each --feature argument there should be one --version argument.") - } + if (numFailures > 0) { + throw new TerseFailure(s"${numFailures} out of ${updates.size} operation(s) failed.") } - features.zip(versions).map { case (feature, version) => - feature -> version - }.toMap - } - - def defaultFeatures(): Map[String, Short] = { - Map.empty - } - - def featuresForRelease(release: String): Map[String, Short] = { - Map.empty } } diff --git a/core/src/main/scala/kafka/admin/MetadataQuorumCommand.scala b/core/src/main/scala/kafka/admin/MetadataQuorumCommand.scala new file mode 100644 index 0000000000000..c92988d97fa18 --- /dev/null +++ b/core/src/main/scala/kafka/admin/MetadataQuorumCommand.scala @@ -0,0 +1,172 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.admin + +import kafka.tools.TerseFailure +import kafka.utils.Exit +import net.sourceforge.argparse4j.ArgumentParsers +import net.sourceforge.argparse4j.impl.Arguments.{fileType, storeTrue} +import net.sourceforge.argparse4j.inf.Subparsers +import org.apache.kafka.clients._ +import org.apache.kafka.clients.admin.{Admin, QuorumInfo} +import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.util.ToolsUtils.prettyPrintTable + +import java.io.File +import java.util.Properties +import scala.jdk.CollectionConverters._ + +/** + * A tool for describing quorum status + */ +object MetadataQuorumCommand { + + def main(args: Array[String]): Unit = { + val res = mainNoExit(args) + Exit.exit(res) + } + + def mainNoExit(args: Array[String]): Int = { + val parser = ArgumentParsers + .newArgumentParser("kafka-metadata-quorum") + .defaultHelp(true) + .description("This tool describes kraft metadata quorum status.") + parser + .addArgument("--bootstrap-server") + .help("A comma-separated list of host:port pairs to use for establishing the connection to the Kafka cluster.") + .required(true) + + parser + .addArgument("--command-config") + .`type`(fileType()) + .help("Property file containing configs to be passed to Admin Client.") + val subparsers = parser.addSubparsers().dest("command") + addDescribeParser(subparsers) + + var admin: Admin = null + try { + val namespace = parser.parseArgsOrFail(args) + val command = namespace.getString("command") + + val commandConfig = namespace.get[File]("command_config") + val props = if (commandConfig != null) { + if (!commandConfig.exists()) { + throw new TerseFailure(s"Properties file ${commandConfig.getPath} does not exists!") + } + Utils.loadProps(commandConfig.getPath) + } else { + new Properties() + } + props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, namespace.getString("bootstrap_server")) + admin = Admin.create(props) + + if (command == "describe") { + if (namespace.getBoolean("status") && namespace.getBoolean("replication")) { + throw new TerseFailure(s"Only one of --status or --replication should be specified with describe sub-command") + } else if (namespace.getBoolean("replication")) { + handleDescribeReplication(admin) + } else if (namespace.getBoolean("status")) { + handleDescribeStatus(admin) + } else { + throw new TerseFailure(s"One of --status or --replication must be specified with describe sub-command") + } + } else { + throw new IllegalStateException(s"Unknown command: $command, only 'describe' is supported") + } + 0 + } catch { + case e: TerseFailure => + Console.err.println(e.getMessage) + 1 + } finally { + if (admin != null) { + admin.close() + } + } + } + + def addDescribeParser(subparsers: Subparsers): Unit = { + val describeParser = subparsers + .addParser("describe") + .help("Describe the metadata quorum info") + + val statusArgs = describeParser.addArgumentGroup("Status") + statusArgs + .addArgument("--status") + .help( + "A short summary of the quorum status and the other provides detailed information about the status of replication.") + .action(storeTrue()) + val replicationArgs = describeParser.addArgumentGroup("Replication") + replicationArgs + .addArgument("--replication") + .help("Detailed information about the status of replication") + .action(storeTrue()) + } + + private def handleDescribeReplication(admin: Admin): Unit = { + val quorumInfo = admin.describeMetadataQuorum.quorumInfo.get + val leaderId = quorumInfo.leaderId + val leader = quorumInfo.voters.asScala.filter(_.replicaId == leaderId).head + + def convertQuorumInfo(infos: Seq[QuorumInfo.ReplicaState], status: String): Seq[Array[String]] = + infos.map { info => + Array(info.replicaId, + info.logEndOffset, + leader.logEndOffset - info.logEndOffset, + info.lastFetchTimestamp.orElse(-1), + info.lastCaughtUpTimestamp.orElse(-1), + status + ).map(_.toString) + } + prettyPrintTable( + Array("NodeId", "LogEndOffset", "Lag", "LastFetchTimestamp", "LastCaughtUpTimestamp", "Status"), + (convertQuorumInfo(Seq(leader), "Leader") + ++ convertQuorumInfo(quorumInfo.voters.asScala.filter(_.replicaId != leaderId).toSeq, "Follower") + ++ convertQuorumInfo(quorumInfo.observers.asScala.toSeq, "Observer")).asJava, + scala.Console.out + ) + } + + private def handleDescribeStatus(admin: Admin): Unit = { + val clusterId = admin.describeCluster.clusterId.get + val quorumInfo = admin.describeMetadataQuorum.quorumInfo.get + val leaderId = quorumInfo.leaderId + val leader = quorumInfo.voters.asScala.filter(_.replicaId == leaderId).head + val maxLagFollower = quorumInfo.voters.asScala + .minBy(_.logEndOffset) + val maxFollowerLag = leader.logEndOffset - maxLagFollower.logEndOffset + val maxFollowerLagTimeMs = + if (leader == maxLagFollower) { + 0 + } else if (leader.lastCaughtUpTimestamp.isPresent && maxLagFollower.lastCaughtUpTimestamp.isPresent) { + leader.lastCaughtUpTimestamp.getAsLong - maxLagFollower.lastCaughtUpTimestamp.getAsLong + } else { + -1 + } + println( + s"""|ClusterId: $clusterId + |LeaderId: ${quorumInfo.leaderId} + |LeaderEpoch: ${quorumInfo.leaderEpoch} + |HighWatermark: ${quorumInfo.highWatermark} + |MaxFollowerLag: $maxFollowerLag + |MaxFollowerLagTimeMs: $maxFollowerLagTimeMs + |CurrentVoters: ${quorumInfo.voters.asScala.map(_.replicaId).mkString("[", ",", "]")} + |CurrentObservers: ${quorumInfo.observers.asScala.map(_.replicaId).mkString("[", ",", "]")} + |""".stripMargin + ) + } +} diff --git a/core/src/main/scala/kafka/api/LeaderAndIsr.scala b/core/src/main/scala/kafka/api/LeaderAndIsr.scala index dd1d381a14455..da68cdb479c1c 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsr.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsr.scala @@ -62,7 +62,7 @@ case class LeaderAndIsr( if (leader == LeaderAndIsr.NoLeader) None else Some(leader) } - def equalsIgnorePartitionEpoch(other: LeaderAndIsr): Boolean = { + def equalsAllowStalePartitionEpoch(other: LeaderAndIsr): Boolean = { if (this == other) { true } else if (other == null) { @@ -71,7 +71,8 @@ case class LeaderAndIsr( leader == other.leader && leaderEpoch == other.leaderEpoch && isr.equals(other.isr) && - leaderRecoveryState == other.leaderRecoveryState + leaderRecoveryState == other.leaderRecoveryState && + partitionEpoch <= other.partitionEpoch } } diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 319025226c7e1..e1d2d85578363 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -26,7 +26,7 @@ import kafka.log._ import kafka.metrics.KafkaMetricsGroup import kafka.server._ import kafka.server.checkpoints.OffsetCheckpoints -import kafka.server.metadata.KRaftMetadataCache +import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache} import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils._ import kafka.zookeeper.ZooKeeperClientException @@ -373,7 +373,7 @@ class Partition(val topicPartition: TopicPartition, // Visible for testing private[cluster] def createLog(isNew: Boolean, isFutureReplica: Boolean, offsetCheckpoints: OffsetCheckpoints, topicId: Option[Uuid]): UnifiedLog = { - def updateHighWatermark(log: UnifiedLog) = { + def updateHighWatermark(log: UnifiedLog): Unit = { val checkpointHighWatermark = offsetCheckpoints.fetch(log.parentDir, topicPartition).getOrElse { info(s"No checkpointed highwatermark is found for partition $topicPartition") 0L @@ -881,11 +881,16 @@ class Partition(val topicPartition: TopicPartition, private def isReplicaIsrEligible(followerReplicaId: Int): Boolean = { metadataCache match { // In KRaft mode, only replicas which are not fenced nor in controlled shutdown are - // allowed to join the ISR. This does not apply to ZK mode. + // allowed to join the ISR. case kRaftMetadataCache: KRaftMetadataCache => !kRaftMetadataCache.isBrokerFenced(followerReplicaId) && !kRaftMetadataCache.isBrokerShuttingDown(followerReplicaId) + // In ZK mode, we just ensure the broker is alive. Although we do not check for shutting down brokers here, + // the controller will block them from being added to ISR. + case zkMetadataCache: ZkMetadataCache => + zkMetadataCache.hasAliveBroker(followerReplicaId) + case _ => true } } @@ -1205,22 +1210,32 @@ class Partition(val topicPartition: TopicPartition, minOneMessage: Boolean, updateFetchState: Boolean ): LogReadInfo = { - def readFromLocalLog(): LogReadInfo = { + def readFromLocalLog(log: UnifiedLog): LogReadInfo = { readRecords( + log, fetchPartitionData.lastFetchedEpoch, fetchPartitionData.fetchOffset, fetchPartitionData.currentLeaderEpoch, maxBytes, fetchParams.isolation, - minOneMessage, - fetchParams.fetchOnlyLeader + minOneMessage ) } if (fetchParams.isFromFollower) { // Check that the request is from a valid replica before doing the read - val replica = followerReplicaOrThrow(fetchParams.replicaId, fetchPartitionData) - val logReadInfo = readFromLocalLog() + val (replica, logReadInfo) = inReadLock(leaderIsrUpdateLock) { + val localLog = localLogWithEpochOrThrow( + fetchPartitionData.currentLeaderEpoch, + fetchParams.fetchOnlyLeader + ) + val replica = followerReplicaOrThrow( + fetchParams.replicaId, + fetchPartitionData + ) + val logReadInfo = readFromLocalLog(localLog) + (replica, logReadInfo) + } if (updateFetchState && logReadInfo.divergingEpoch.isEmpty) { updateFollowerFetchState( @@ -1234,7 +1249,13 @@ class Partition(val topicPartition: TopicPartition, logReadInfo } else { - readFromLocalLog() + inReadLock(leaderIsrUpdateLock) { + val localLog = localLogWithEpochOrThrow( + fetchPartitionData.currentLeaderEpoch, + fetchParams.fetchOnlyLeader + ) + readFromLocalLog(localLog) + } } } @@ -1270,16 +1291,14 @@ class Partition(val topicPartition: TopicPartition, } private def readRecords( + localLog: UnifiedLog, lastFetchedEpoch: Optional[Integer], fetchOffset: Long, currentLeaderEpoch: Optional[Integer], maxBytes: Int, fetchIsolation: FetchIsolation, - minOneMessage: Boolean, - fetchOnlyFromLeader: Boolean - ): LogReadInfo = inReadLock(leaderIsrUpdateLock) { - val localLog = localLogWithEpochOrThrow(currentLeaderEpoch, fetchOnlyFromLeader) - + minOneMessage: Boolean + ): LogReadInfo = { // Note we use the log end offset prior to the read. This ensures that any appends following // the fetch do not prevent a follower from coming into sync. val initialHighWatermark = localLog.highWatermark diff --git a/core/src/main/scala/kafka/controller/ControllerContext.scala b/core/src/main/scala/kafka/controller/ControllerContext.scala index 379196aa1d42c..7065d87c4c606 100644 --- a/core/src/main/scala/kafka/controller/ControllerContext.scala +++ b/core/src/main/scala/kafka/controller/ControllerContext.scala @@ -327,9 +327,16 @@ class ControllerContext { } } - def queueTopicDeletion(topics: Set[String]): Unit = { - topicsToBeDeleted ++= topics - topics.foreach(cleanPreferredReplicaImbalanceMetric) + def queueTopicDeletion(topicToBeAddedIntoDeletionList: Set[String]): Unit = { + // queueTopicDeletion could be called multiple times for same topic. + // e.g. 1) delete topic-A => 2) delete topic-B before A's deletion completes. + // In this case, at 2), queueTopicDeletion will be called with Set(topic-A, topic-B). + // However we should call cleanPreferredReplicaImbalanceMetric only once for same topic + // because otherwise, preferredReplicaImbalanceCount could be decremented wrongly at 2nd call. + // So we need to take a diff with already queued topics here. + val newlyDeletedTopics = topicToBeAddedIntoDeletionList.diff(topicsToBeDeleted) + topicsToBeDeleted ++= newlyDeletedTopics + newlyDeletedTopics.foreach(cleanPreferredReplicaImbalanceMetric) } def beginTopicDeletion(topics: Set[String]): Unit = { diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 8d16eb7e1da8c..999bcb818e91c 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -2336,17 +2336,25 @@ class KafkaController(val config: KafkaConfig, controllerContext.partitionLeadershipInfo(tp) match { case Some(leaderIsrAndControllerEpoch) => val currentLeaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr - if (newLeaderAndIsr.leaderEpoch != currentLeaderAndIsr.leaderEpoch) { - partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH) + if (newLeaderAndIsr.partitionEpoch > currentLeaderAndIsr.partitionEpoch + || newLeaderAndIsr.leaderEpoch > currentLeaderAndIsr.leaderEpoch) { + // If the partition leader has a higher partition/leader epoch, then it is likely + // that this node is no longer the active controller. We return NOT_CONTROLLER in + // this case to give the leader an opportunity to find the new controller. + partitionResponses(tp) = Left(Errors.NOT_CONTROLLER) None - } else if (newLeaderAndIsr.partitionEpoch < currentLeaderAndIsr.partitionEpoch) { - partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION) + } else if (newLeaderAndIsr.leaderEpoch != currentLeaderAndIsr.leaderEpoch) { + partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH) None - } else if (newLeaderAndIsr.equalsIgnorePartitionEpoch(currentLeaderAndIsr)) { + } else if (newLeaderAndIsr.equalsAllowStalePartitionEpoch(currentLeaderAndIsr)) { // If a partition is already in the desired state, just return it + // this check must be done before fencing based on partition epoch to maintain idempotency partitionResponses(tp) = Right(currentLeaderAndIsr) None - } else if (newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) { + } else if (newLeaderAndIsr.partitionEpoch != currentLeaderAndIsr.partitionEpoch) { + partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION) + None + } else if (newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) { partitionResponses(tp) = Left(Errors.INVALID_REQUEST) info( s"Rejecting AlterPartition from node $brokerId for $tp because leader is recovering and ISR is greater than 1: " + @@ -2363,7 +2371,23 @@ class KafkaController(val config: KafkaConfig, ) None } else { - Some(tp -> newLeaderAndIsr) + // Pull out replicas being added to ISR and verify they are all online. + // If a replica is not online, reject the update as specified in KIP-841. + val ineligibleReplicas = newLeaderAndIsr.isr.toSet -- controllerContext.liveBrokerIds + if (ineligibleReplicas.nonEmpty) { + info(s"Rejecting AlterPartition request from node $brokerId for $tp because " + + s"it specified ineligible replicas $ineligibleReplicas in the new ISR ${newLeaderAndIsr.isr}." + ) + + if (alterPartitionRequestVersion > 1) { + partitionResponses(tp) = Left(Errors.INELIGIBLE_REPLICA) + } else { + partitionResponses(tp) = Left(Errors.OPERATION_NOT_ATTEMPTED) + } + None + } else { + Some(tp -> newLeaderAndIsr) + } } case None => diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala index 6bf337d6799fe..86c1f174339d2 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala @@ -316,6 +316,8 @@ class GroupCoordinator(val brokerId: Int, newMemberId, groupInstanceId, protocols, + rebalanceTimeoutMs, + sessionTimeoutMs, responseCallback, requestLocal, reason, @@ -438,7 +440,7 @@ class GroupCoordinator(val brokerId: Int, case None => group.currentState match { case PreparingRebalance => val member = group.get(memberId) - updateMemberAndRebalance(group, member, protocols, s"Member ${member.memberId} joining group during ${group.currentState}; client reason: $reason", responseCallback) + updateMemberAndRebalance(group, member, protocols, rebalanceTimeoutMs, sessionTimeoutMs, s"Member ${member.memberId} joining group during ${group.currentState}; client reason: $reason", responseCallback) case CompletingRebalance => val member = group.get(memberId) @@ -461,7 +463,7 @@ class GroupCoordinator(val brokerId: Int, error = Errors.NONE)) } else { // member has changed metadata, so force a rebalance - updateMemberAndRebalance(group, member, protocols, s"Updating metadata for member ${member.memberId} during ${group.currentState}; client reason: $reason", responseCallback) + updateMemberAndRebalance(group, member, protocols, rebalanceTimeoutMs, sessionTimeoutMs, s"Updating metadata for member ${member.memberId} during ${group.currentState}; client reason: $reason", responseCallback) } case Stable => @@ -470,9 +472,9 @@ class GroupCoordinator(val brokerId: Int, // force a rebalance if the leader sends JoinGroup; // This allows the leader to trigger rebalances for changes affecting assignment // which do not affect the member metadata (such as topic metadata changes for the consumer) - updateMemberAndRebalance(group, member, protocols, s"Leader ${member.memberId} re-joining group during ${group.currentState}; client reason: $reason", responseCallback) + updateMemberAndRebalance(group, member, protocols, rebalanceTimeoutMs, sessionTimeoutMs, s"Leader ${member.memberId} re-joining group during ${group.currentState}; client reason: $reason", responseCallback) } else if (!member.matches(protocols)) { - updateMemberAndRebalance(group, member, protocols, s"Updating metadata for member ${member.memberId} during ${group.currentState}; client reason: $reason", responseCallback) + updateMemberAndRebalance(group, member, protocols, rebalanceTimeoutMs, sessionTimeoutMs, s"Updating metadata for member ${member.memberId} during ${group.currentState}; client reason: $reason", responseCallback) } else { // for followers with no actual change to their metadata, just return group information // for the current generation which will allow them to issue SyncGroup @@ -982,6 +984,11 @@ class GroupCoordinator(val brokerId: Int, ): Option[Errors] = { if (group.is(Dead)) { Some(Errors.COORDINATOR_NOT_AVAILABLE) + } else if (generationId < 0 && group.is(Empty)) { + // When the generation id is -1, the request comes from either the admin client + // or a consumer which does not use the group management facility. In this case, + // the request can commit offsets if the group is empty. + None } else if (generationId >= 0 || memberId != JoinGroupRequest.UNKNOWN_MEMBER_ID || groupInstanceId.isDefined) { validateCurrentMember( group, @@ -1288,6 +1295,8 @@ class GroupCoordinator(val brokerId: Int, newMemberId: String, groupInstanceId: String, protocols: List[(String, Array[Byte])], + rebalanceTimeoutMs: Int, + sessionTimeoutMs: Int, responseCallback: JoinCallback, requestLocal: RequestLocal, reason: String, @@ -1300,7 +1309,9 @@ class GroupCoordinator(val brokerId: Int, completeAndScheduleNextHeartbeatExpiration(group, member) val knownStaticMember = group.get(newMemberId) - group.updateMember(knownStaticMember, protocols, responseCallback) + val oldRebalanceTimeoutMs = knownStaticMember.rebalanceTimeoutMs + val oldSessionTimeoutMs = knownStaticMember.sessionTimeoutMs + group.updateMember(knownStaticMember, protocols, rebalanceTimeoutMs, sessionTimeoutMs, responseCallback) val oldProtocols = knownStaticMember.supportedProtocols group.currentState match { @@ -1316,7 +1327,7 @@ class GroupCoordinator(val brokerId: Int, warn(s"Failed to persist metadata for group ${group.groupId}: ${error.message}") // Failed to persist member.id of the given static member, revert the update of the static member in the group. - group.updateMember(knownStaticMember, oldProtocols, null) + group.updateMember(knownStaticMember, oldProtocols, oldRebalanceTimeoutMs, oldSessionTimeoutMs, null) val oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId) completeAndScheduleNextHeartbeatExpiration(group, oldMember) responseCallback(JoinGroupResult( @@ -1395,9 +1406,11 @@ class GroupCoordinator(val brokerId: Int, private def updateMemberAndRebalance(group: GroupMetadata, member: MemberMetadata, protocols: List[(String, Array[Byte])], + rebalanceTimeoutMs: Int, + sessionTimeoutMs: Int, reason: String, callback: JoinCallback): Unit = { - group.updateMember(member, protocols, callback) + group.updateMember(member, protocols, rebalanceTimeoutMs, sessionTimeoutMs, callback) maybePrepareRebalance(group, reason) } diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala index a7a6e915d99ca..b5b0b2df2714e 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala @@ -526,10 +526,14 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState def updateMember(member: MemberMetadata, protocols: List[(String, Array[Byte])], + rebalanceTimeoutMs: Int, + sessionTimeoutMs: Int, callback: JoinCallback): Unit = { decSupportedProtocols(member) member.supportedProtocols = protocols incSupportedProtocols(member) + member.rebalanceTimeoutMs = rebalanceTimeoutMs + member.sessionTimeoutMs = sessionTimeoutMs if (callback != null && !member.isAwaitingJoin) { numMembersAwaitingJoin += 1 @@ -542,9 +546,16 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState def maybeInvokeJoinCallback(member: MemberMetadata, joinGroupResult: JoinGroupResult): Unit = { if (member.isAwaitingJoin) { - member.awaitingJoinCallback(joinGroupResult) - member.awaitingJoinCallback = null - numMembersAwaitingJoin -= 1 + try { + member.awaitingJoinCallback(joinGroupResult) + } catch { + case t: Throwable => + error(s"Failed to invoke join callback for $member due to ${t.getMessage}.", t) + member.awaitingJoinCallback(JoinGroupResult(member.memberId, Errors.UNKNOWN_SERVER_ERROR)) + } finally { + member.awaitingJoinCallback = null + numMembersAwaitingJoin -= 1 + } } } @@ -554,8 +565,15 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState def maybeInvokeSyncCallback(member: MemberMetadata, syncGroupResult: SyncGroupResult): Boolean = { if (member.isAwaitingSync) { - member.awaitingSyncCallback(syncGroupResult) - member.awaitingSyncCallback = null + try { + member.awaitingSyncCallback(syncGroupResult) + } catch { + case t: Throwable => + error(s"Failed to invoke sync callback for $member due to ${t.getMessage}.", t) + member.awaitingSyncCallback(SyncGroupResult(Errors.UNKNOWN_SERVER_ERROR)) + } finally { + member.awaitingSyncCallback = null + } true } else { false diff --git a/core/src/main/scala/kafka/coordinator/group/MemberMetadata.scala b/core/src/main/scala/kafka/coordinator/group/MemberMetadata.scala index 514dbfbd3887d..c94fcf7be4184 100644 --- a/core/src/main/scala/kafka/coordinator/group/MemberMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/group/MemberMetadata.scala @@ -57,8 +57,8 @@ private[group] class MemberMetadata(var memberId: String, val groupInstanceId: Option[String], val clientId: String, val clientHost: String, - val rebalanceTimeoutMs: Int, - val sessionTimeoutMs: Int, + var rebalanceTimeoutMs: Int, + var sessionTimeoutMs: Int, val protocolType: String, var supportedProtocols: List[(String, Array[Byte])], var assignment: Array[Byte] = Array.empty[Byte]) { diff --git a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala index e1f46eb37120c..f16785a7b6c09 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala @@ -167,7 +167,9 @@ class RPCProducerIdManager(brokerId: Int, if (nextProducerId > currentProducerIdBlock.lastProducerId) { val block = nextProducerIdBlock.poll(maxWaitMs, TimeUnit.MILLISECONDS) if (block == null) { - throw Errors.REQUEST_TIMED_OUT.exception("Timed out waiting for next producer ID block") + // Return COORDINATOR_LOAD_IN_PROGRESS rather than REQUEST_TIMED_OUT since older clients treat the error as fatal + // when it should be retriable like COORDINATOR_LOAD_IN_PROGRESS. + throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception("Timed out waiting for next producer ID block") } else { block match { case Success(nextBlock) => @@ -236,7 +238,6 @@ class RPCProducerIdManager(brokerId: Int, private[transaction] def handleTimeout(): Unit = { warn("Timed out when requesting AllocateProducerIds from the controller.") requestInFlight.set(false) - nextProducerIdBlock.put(Failure(Errors.REQUEST_TIMED_OUT.exception)) maybeRequestNextBlock() } } diff --git a/core/src/main/scala/kafka/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala index 581d016e5e0e9..f8da67656fb97 100644 --- a/core/src/main/scala/kafka/log/LogLoader.scala +++ b/core/src/main/scala/kafka/log/LogLoader.scala @@ -19,7 +19,6 @@ package kafka.log import java.io.{File, IOException} import java.nio.file.{Files, NoSuchFileException} - import kafka.common.LogSegmentOffsetOverflowException import kafka.log.UnifiedLog.{CleanedFileSuffix, DeletedFileSuffix, SwapFileSuffix, isIndexFile, isLogFile, offsetFromFile} import kafka.server.{LogDirFailureChannel, LogOffsetMetadata} @@ -28,7 +27,9 @@ import kafka.utils.{CoreUtils, Logging, Scheduler} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.InvalidOffsetException import org.apache.kafka.common.utils.Time +import org.apache.kafka.snapshot.Snapshots +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import scala.collection.{Set, mutable} case class LoadedLogOffsets(logStartOffset: Long, @@ -64,6 +65,7 @@ object LogLoader extends Logging { * @param recoveryPointCheckpoint The checkpoint of the offset at which to begin the recovery * @param leaderEpochCache An optional LeaderEpochFileCache instance to be updated during recovery * @param producerStateManager The ProducerStateManager instance to be updated during recovery + * @param numRemainingSegments The remaining segments to be recovered in this log keyed by recovery thread name */ class LogLoader( dir: File, @@ -77,7 +79,8 @@ class LogLoader( logStartOffsetCheckpoint: Long, recoveryPointCheckpoint: Long, leaderEpochCache: Option[LeaderEpochFileCache], - producerStateManager: ProducerStateManager + producerStateManager: ProducerStateManager, + numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int] ) extends Logging { logIdent = s"[LogLoader partition=$topicPartition, dir=${dir.getParent}] " @@ -226,7 +229,10 @@ class LogLoader( if (!file.canRead) throw new IOException(s"Could not read file $file") val filename = file.getName - if (filename.endsWith(DeletedFileSuffix)) { + + // Delete stray files marked for deletion, but skip KRaft snapshots. + // These are handled in the recovery logic in `KafkaMetadataLog`. + if (filename.endsWith(DeletedFileSuffix) && !filename.endsWith(Snapshots.DELETE_SUFFIX)) { debug(s"Deleting stray temporary file ${file.getAbsolutePath}") Files.deleteIfExists(file.toPath) } else if (filename.endsWith(CleanedFileSuffix)) { @@ -404,12 +410,18 @@ class LogLoader( // If we have the clean shutdown marker, skip recovery. if (!hadCleanShutdown) { - val unflushed = segments.values(recoveryPointCheckpoint, Long.MaxValue).iterator + val unflushed = segments.values(recoveryPointCheckpoint, Long.MaxValue) + val numUnflushed = unflushed.size + val unflushedIter = unflushed.iterator var truncated = false + var numFlushed = 0 + val threadName = Thread.currentThread().getName + numRemainingSegments.put(threadName, numUnflushed) + + while (unflushedIter.hasNext && !truncated) { + val segment = unflushedIter.next() + info(s"Recovering unflushed segment ${segment.baseOffset}. $numFlushed/$numUnflushed recovered for $topicPartition.") - while (unflushed.hasNext && !truncated) { - val segment = unflushed.next() - info(s"Recovering unflushed segment ${segment.baseOffset}") val truncatedBytes = try { recoverSegment(segment) @@ -424,8 +436,13 @@ class LogLoader( // we had an invalid message, delete all remaining log warn(s"Corruption found in segment ${segment.baseOffset}," + s" truncating to offset ${segment.readNextOffset}") - removeAndDeleteSegmentsAsync(unflushed.toList) + removeAndDeleteSegmentsAsync(unflushedIter.toList) truncated = true + // segment is truncated, so set remaining segments to 0 + numRemainingSegments.put(threadName, 0) + } else { + numFlushed += 1 + numRemainingSegments.put(threadName, numUnflushed - numFlushed) } } } diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index bdc7ffd74dbea..886f56c63cfa9 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -262,7 +262,8 @@ class LogManager(logDirs: Seq[File], recoveryPoints: Map[TopicPartition, Long], logStartOffsets: Map[TopicPartition, Long], defaultConfig: LogConfig, - topicConfigOverrides: Map[String, LogConfig]): UnifiedLog = { + topicConfigOverrides: Map[String, LogConfig], + numRemainingSegments: ConcurrentMap[String, Int]): UnifiedLog = { val topicPartition = UnifiedLog.parseTopicPartitionName(logDir) val config = topicConfigOverrides.getOrElse(topicPartition.topic, defaultConfig) val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) @@ -282,7 +283,8 @@ class LogManager(logDirs: Seq[File], logDirFailureChannel = logDirFailureChannel, lastShutdownClean = hadCleanShutdown, topicId = None, - keepPartitionMetadataFile = keepPartitionMetadataFile) + keepPartitionMetadataFile = keepPartitionMetadataFile, + numRemainingSegments = numRemainingSegments) if (logDir.getName.endsWith(UnifiedLog.DeleteDirSuffix)) { addLogToBeDeleted(log) @@ -307,6 +309,27 @@ class LogManager(logDirs: Seq[File], log } + // factory class for naming the log recovery threads used in metrics + class LogRecoveryThreadFactory(val dirPath: String) extends ThreadFactory { + val threadNum = new AtomicInteger(0) + + override def newThread(runnable: Runnable): Thread = { + KafkaThread.nonDaemon(logRecoveryThreadName(dirPath, threadNum.getAndIncrement()), runnable) + } + } + + // create a unique log recovery thread name for each log dir as the format: prefix-dirPath-threadNum, ex: "log-recovery-/tmp/kafkaLogs-0" + private def logRecoveryThreadName(dirPath: String, threadNum: Int, prefix: String = "log-recovery"): String = s"$prefix-$dirPath-$threadNum" + + /* + * decrement the number of remaining logs + * @return the number of remaining logs after decremented 1 + */ + private[log] def decNumRemainingLogs(numRemainingLogs: ConcurrentMap[String, Int], path: String): Int = { + require(path != null, "path cannot be null to update remaining logs metric.") + numRemainingLogs.compute(path, (_, oldVal) => oldVal - 1) + } + /** * Recover and load all logs in the given data directories */ @@ -317,6 +340,10 @@ class LogManager(logDirs: Seq[File], val offlineDirs = mutable.Set.empty[(String, IOException)] val jobs = ArrayBuffer.empty[Seq[Future[_]]] var numTotalLogs = 0 + // log dir path -> number of Remaining logs map for remainingLogsToRecover metric + val numRemainingLogs: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int] + // log recovery thread name -> number of remaining segments map for remainingSegmentsToRecover metric + val numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int] def handleIOException(logDirAbsolutePath: String, e: IOException): Unit = { offlineDirs.add((logDirAbsolutePath, e)) @@ -328,7 +355,7 @@ class LogManager(logDirs: Seq[File], var hadCleanShutdown: Boolean = false try { val pool = Executors.newFixedThreadPool(numRecoveryThreadsPerDataDir, - KafkaThread.nonDaemon(s"log-recovery-$logDirAbsolutePath", _)) + new LogRecoveryThreadFactory(logDirAbsolutePath)) threadPools.append(pool) val cleanShutdownFile = new File(dir, LogLoader.CleanShutdownFile) @@ -363,28 +390,32 @@ class LogManager(logDirs: Seq[File], val logsToLoad = Option(dir.listFiles).getOrElse(Array.empty).filter(logDir => logDir.isDirectory && UnifiedLog.parseTopicPartitionName(logDir).topic != KafkaRaftServer.MetadataTopic) - val numLogsLoaded = new AtomicInteger(0) numTotalLogs += logsToLoad.length + numRemainingLogs.put(dir.getAbsolutePath, logsToLoad.length) val jobsForDir = logsToLoad.map { logDir => val runnable: Runnable = () => { + debug(s"Loading log $logDir") + var log = None: Option[UnifiedLog] + val logLoadStartMs = time.hiResClockMs() try { - debug(s"Loading log $logDir") - - val logLoadStartMs = time.hiResClockMs() - val log = loadLog(logDir, hadCleanShutdown, recoveryPoints, logStartOffsets, - defaultConfig, topicConfigOverrides) - val logLoadDurationMs = time.hiResClockMs() - logLoadStartMs - val currentNumLoaded = numLogsLoaded.incrementAndGet() - - info(s"Completed load of $log with ${log.numberOfSegments} segments in ${logLoadDurationMs}ms " + - s"($currentNumLoaded/${logsToLoad.length} loaded in $logDirAbsolutePath)") + log = Some(loadLog(logDir, hadCleanShutdown, recoveryPoints, logStartOffsets, + defaultConfig, topicConfigOverrides, numRemainingSegments)) } catch { case e: IOException => handleIOException(logDirAbsolutePath, e) case e: KafkaStorageException if e.getCause.isInstanceOf[IOException] => // KafkaStorageException might be thrown, ex: during writing LeaderEpochFileCache // And while converting IOException to KafkaStorageException, we've already handled the exception. So we can ignore it here. + } finally { + val logLoadDurationMs = time.hiResClockMs() - logLoadStartMs + val remainingLogs = decNumRemainingLogs(numRemainingLogs, dir.getAbsolutePath) + val currentNumLoaded = logsToLoad.length - remainingLogs + log match { + case Some(loadedLog) => info(s"Completed load of $loadedLog with ${loadedLog.numberOfSegments} segments in ${logLoadDurationMs}ms " + + s"($currentNumLoaded/${logsToLoad.length} completed in $logDirAbsolutePath)") + case None => info(s"Error while loading logs in $logDir in ${logLoadDurationMs}ms ($currentNumLoaded/${logsToLoad.length} completed in $logDirAbsolutePath)") + } } } runnable @@ -398,6 +429,7 @@ class LogManager(logDirs: Seq[File], } try { + addLogRecoveryMetrics(numRemainingLogs, numRemainingSegments) for (dirJobs <- jobs) { dirJobs.foreach(_.get) } @@ -410,12 +442,37 @@ class LogManager(logDirs: Seq[File], error(s"There was an error in one of the threads during logs loading: ${e.getCause}") throw e.getCause } finally { + removeLogRecoveryMetrics() threadPools.foreach(_.shutdown()) } info(s"Loaded $numTotalLogs logs in ${time.hiResClockMs() - startMs}ms.") } + private[log] def addLogRecoveryMetrics(numRemainingLogs: ConcurrentMap[String, Int], + numRemainingSegments: ConcurrentMap[String, Int]): Unit = { + debug("Adding log recovery metrics") + for (dir <- logDirs) { + newGauge("remainingLogsToRecover", () => numRemainingLogs.get(dir.getAbsolutePath), + Map("dir" -> dir.getAbsolutePath)) + for (i <- 0 until numRecoveryThreadsPerDataDir) { + val threadName = logRecoveryThreadName(dir.getAbsolutePath, i) + newGauge("remainingSegmentsToRecover", () => numRemainingSegments.get(threadName), + Map("dir" -> dir.getAbsolutePath, "threadNum" -> i.toString)) + } + } + } + + private[log] def removeLogRecoveryMetrics(): Unit = { + debug("Removing log recovery metrics") + for (dir <- logDirs) { + removeMetric("remainingLogsToRecover", Map("dir" -> dir.getAbsolutePath)) + for (i <- 0 until numRecoveryThreadsPerDataDir) { + removeMetric("remainingSegmentsToRecover", Map("dir" -> dir.getAbsolutePath, "threadNum" -> i.toString)) + } + } + } + /** * Start the background threads to flush logs and do log cleanup */ diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index ddd66eb160fcb..e976244833cb2 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -18,11 +18,11 @@ package kafka.log import com.yammer.metrics.core.MetricName + import java.io.{File, IOException} import java.nio.file.Files import java.util.Optional -import java.util.concurrent.TimeUnit - +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, TimeUnit} import kafka.common.{LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException} import kafka.log.AppendOrigin.RaftLeader import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec} @@ -373,8 +373,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, * Update the high watermark to a new offset. The new high watermark will be lower * bounded by the log start offset and upper bounded by the log end offset. * - * This is intended to be called when initializing the high watermark or when updating - * it on a follower after receiving a Fetch response from the leader. + * This is intended to be called by the leader when initializing the high watermark. * * @param hw the suggested new value for the high watermark * @return the updated high watermark offset @@ -433,6 +432,27 @@ class UnifiedLog(@volatile var logStartOffset: Long, } } + /** + * Update high watermark with a new value. The new high watermark will be lower + * bounded by the log start offset and upper bounded by the log end offset. + * + * This method is intended to be used by the follower to update its high watermark after + * replication from the leader. + * + * @return the new high watermark if the high watermark changed, None otherwise. + */ + def maybeUpdateHighWatermark(hw: Long): Option[Long] = { + lock.synchronized { + val oldHighWatermark = highWatermarkMetadata + updateHighWatermark(LogOffsetMetadata(hw)) match { + case oldHighWatermark.messageOffset => + None + case newHighWatermark => + Some(newHighWatermark) + } + } + } + /** * Get the offset and metadata for the current high watermark. If offset metadata is not * known, this will do a lookup in the index and cache the result. @@ -1803,7 +1823,8 @@ object UnifiedLog extends Logging { logDirFailureChannel: LogDirFailureChannel, lastShutdownClean: Boolean = true, topicId: Option[Uuid], - keepPartitionMetadataFile: Boolean): UnifiedLog = { + keepPartitionMetadataFile: Boolean, + numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int]): UnifiedLog = { // create the log directory if it doesn't exist Files.createDirectories(dir.toPath) val topicPartition = UnifiedLog.parseTopicPartitionName(dir) @@ -1828,7 +1849,8 @@ object UnifiedLog extends Logging { logStartOffset, recoveryPoint, leaderEpochCache, - producerStateManager + producerStateManager, + numRemainingSegments ).load() val localLog = new LocalLog(dir, config, segments, offsets.recoveryPoint, offsets.nextOffsetMetadata, scheduler, time, topicPartition, logDirFailureChannel) diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 4fa611206a4cf..2200757c70655 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -121,10 +121,17 @@ object RequestChannel extends Logging { def isForwarded: Boolean = envelope.isDefined + private def shouldReturnNotController(response: AbstractResponse): Boolean = { + response match { + case describeQuorumResponse: DescribeQuorumResponse => response.errorCounts.containsKey(Errors.NOT_LEADER_OR_FOLLOWER) + case _ => response.errorCounts.containsKey(Errors.NOT_CONTROLLER) + } + } + def buildResponseSend(abstractResponse: AbstractResponse): Send = { envelope match { case Some(request) => - val envelopeResponse = if (abstractResponse.errorCounts().containsKey(Errors.NOT_CONTROLLER)) { + val envelopeResponse = if (shouldReturnNotController(abstractResponse)) { // Since it's a NOT_CONTROLLER error response, we need to make envelope response with NOT_CONTROLLER error // to notify the requester (i.e. BrokerToControllerRequestThread) to update active controller new EnvelopeResponse(new EnvelopeResponseData() diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index dba8975d437ed..95d96b3399d30 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -16,7 +16,7 @@ */ package kafka.raft -import kafka.log.{AppendOrigin, Defaults, UnifiedLog, LogConfig, LogOffsetSnapshot, SnapshotGenerated} +import kafka.log.{AppendOrigin, Defaults, LogConfig, LogOffsetSnapshot, SnapshotGenerated, UnifiedLog} import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, MetadataLogSegmentMinBytesProp} import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchLogEnd, KafkaConfig, LogDirFailureChannel, RequestLocal} import kafka.utils.{CoreUtils, Logging, Scheduler} @@ -26,7 +26,7 @@ import org.apache.kafka.common.record.{ControlRecordUtils, MemoryRecords, Record import org.apache.kafka.common.utils.{BufferSupplier, Time} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, ValidOffsetAndEpoch} -import org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, Snapshots} +import org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} import java.io.File import java.nio.file.{Files, NoSuchFileException, Path} @@ -257,7 +257,11 @@ final class KafkaMetadataLog private ( } override def storeSnapshot(snapshotId: OffsetAndEpoch): Optional[RawSnapshotWriter] = { - if (snapshots.contains(snapshotId)) { + val containsSnapshotId = snapshots synchronized { + snapshots.contains(snapshotId) + } + + if (containsSnapshotId) { Optional.empty() } else { Optional.of(FileRawSnapshotWriter.create(log.dir.toPath, snapshotId, Optional.of(this))) @@ -378,7 +382,7 @@ final class KafkaMetadataLog private ( } /** - * Perform cleaning of old snapshots and log segments based on size. + * Perform cleaning of old snapshots and log segments based on size and time. * * If our configured retention size has been violated, we perform cleaning as follows: * @@ -542,7 +546,7 @@ case class MetadataLogConfig(logSegmentBytes: Int, fileDeleteDelayMs: Int, nodeId: Int) -object KafkaMetadataLog { +object KafkaMetadataLog extends Logging { def apply( topicPartition: TopicPartition, topicId: Uuid, @@ -552,15 +556,29 @@ object KafkaMetadataLog { config: MetadataLogConfig ): KafkaMetadataLog = { val props = new Properties() - props.put(LogConfig.MaxMessageBytesProp, config.maxBatchSizeInBytes.toString) - props.put(LogConfig.SegmentBytesProp, Int.box(config.logSegmentBytes)) - props.put(LogConfig.SegmentMsProp, Long.box(config.logSegmentMillis)) - props.put(LogConfig.FileDeleteDelayMsProp, Int.box(Defaults.FileDeleteDelayMs)) + props.setProperty(LogConfig.MaxMessageBytesProp, config.maxBatchSizeInBytes.toString) + props.setProperty(LogConfig.SegmentBytesProp, config.logSegmentBytes.toString) + props.setProperty(LogConfig.SegmentMsProp, config.logSegmentMillis.toString) + props.setProperty(LogConfig.FileDeleteDelayMsProp, Defaults.FileDeleteDelayMs.toString) + + // Disable time and byte retention when deleting segments + props.setProperty(LogConfig.RetentionMsProp, "-1") + props.setProperty(LogConfig.RetentionBytesProp, "-1") LogConfig.validateValues(props) val defaultLogConfig = LogConfig(props) if (config.logSegmentBytes < config.logSegmentMinBytes) { - throw new InvalidConfigurationException(s"Cannot set $MetadataLogSegmentBytesProp below ${config.logSegmentMinBytes}") + throw new InvalidConfigurationException( + s"Cannot set $MetadataLogSegmentBytesProp below ${config.logSegmentMinBytes}: ${config.logSegmentBytes}" + ) + } else if (defaultLogConfig.retentionMs >= 0) { + throw new InvalidConfigurationException( + s"Cannot set ${LogConfig.RetentionMsProp} above -1: ${defaultLogConfig.retentionMs}." + ) + } else if (defaultLogConfig.retentionSize >= 0) { + throw new InvalidConfigurationException( + s"Cannot set ${LogConfig.RetentionBytesProp} above -1: ${defaultLogConfig.retentionSize}." + ) } val log = UnifiedLog( @@ -605,7 +623,9 @@ object KafkaMetadataLog { private def recoverSnapshots( log: UnifiedLog ): mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]] = { - val snapshots = mutable.TreeMap.empty[OffsetAndEpoch, Option[FileRawSnapshotReader]] + val snapshotsToRetain = mutable.TreeMap.empty[OffsetAndEpoch, Option[FileRawSnapshotReader]] + val snapshotsToDelete = mutable.Buffer.empty[SnapshotPath] + // Scan the log directory; deleting partial snapshots and older snapshot, only remembering immutable snapshots start // from logStartOffset val filesInDir = Files.newDirectoryStream(log.dir.toPath) @@ -613,21 +633,40 @@ object KafkaMetadataLog { try { filesInDir.forEach { path => Snapshots.parse(path).ifPresent { snapshotPath => - if (snapshotPath.partial || - snapshotPath.deleted || - snapshotPath.snapshotId.offset < log.logStartOffset) { - // Delete partial snapshot, deleted snapshot and older snapshot - Files.deleteIfExists(snapshotPath.path) + // Collect partial snapshot, deleted snapshot and older snapshot for deletion + if (snapshotPath.partial + || snapshotPath.deleted + || snapshotPath.snapshotId.offset < log.logStartOffset) { + snapshotsToDelete.append(snapshotPath) } else { - snapshots.put(snapshotPath.snapshotId, None) + snapshotsToRetain.put(snapshotPath.snapshotId, None) } } } + + // Before deleting any snapshots, we should ensure that the retained snapshots are + // consistent with the current state of the log. If the log start offset is not 0, + // then we must have a snapshot which covers the initial state up to the current + // log start offset. + if (log.logStartOffset > 0) { + val latestSnapshotId = snapshotsToRetain.lastOption.map(_._1) + if (!latestSnapshotId.exists(snapshotId => snapshotId.offset >= log.logStartOffset)) { + throw new IllegalStateException("Inconsistent snapshot state: there must be a snapshot " + + s"at an offset larger then the current log start offset ${log.logStartOffset}, but the " + + s"latest snapshot is $latestSnapshotId") + } + } + + snapshotsToDelete.foreach { snapshotPath => + Files.deleteIfExists(snapshotPath.path) + info(s"Deleted unneeded snapshot file with path $snapshotPath") + } } finally { filesInDir.close() } - snapshots + info(s"Initialized snapshots with IDs ${snapshotsToRetain.keys} from ${log.dir}") + snapshotsToRetain } private def deleteSnapshotFiles( diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index cbb9f7b89bf8e..5b8fe1e827699 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -24,7 +24,6 @@ import java.util.concurrent.CompletableFuture import kafka.log.UnifiedLog import kafka.raft.KafkaRaftManager.RaftIoThread import kafka.server.{KafkaConfig, MetaProperties} -import kafka.server.KafkaRaftServer.ControllerRole import kafka.utils.timer.SystemTimer import kafka.utils.{KafkaScheduler, Logging, ShutdownableThread} import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, NetworkClient} @@ -123,6 +122,8 @@ class KafkaRaftManager[T]( private val dataDir = createDataDir() override val replicatedLog: ReplicatedLog = buildMetadataLog() private val netChannel = buildNetworkChannel() + private val expirationTimer = new SystemTimer("raft-expiration-executor") + private val expirationService = new TimingWheelExpirationService(expirationTimer) override val client: KafkaRaftClient[T] = buildRaftClient() private val raftIoThread = new RaftIoThread(client, threadNamePrefix) @@ -134,10 +135,10 @@ class KafkaRaftManager[T]( case spec: InetAddressSpec => netChannel.updateEndpoint(voterAddressEntry.getKey, spec) case _: UnknownAddressSpec => - logger.info(s"Skipping channel update for destination ID: ${voterAddressEntry.getKey} " + + info(s"Skipping channel update for destination ID: ${voterAddressEntry.getKey} " + s"because of non-routable endpoint: ${NON_ROUTABLE_ADDRESS.toString}") case invalid: AddressSpec => - logger.warn(s"Unexpected address spec (type: ${invalid.getClass}) for channel update for " + + warn(s"Unexpected address spec (type: ${invalid.getClass}) for channel update for " + s"destination ID: ${voterAddressEntry.getKey}") } } @@ -146,6 +147,8 @@ class KafkaRaftManager[T]( } def shutdown(): Unit = { + expirationService.shutdown() + expirationTimer.shutdown() raftIoThread.shutdown() client.close() scheduler.shutdown() @@ -178,15 +181,8 @@ class KafkaRaftManager[T]( } private def buildRaftClient(): KafkaRaftClient[T] = { - val expirationTimer = new SystemTimer("raft-expiration-executor") - val expirationService = new TimingWheelExpirationService(expirationTimer) val quorumStateStore = new FileBasedStateStore(new File(dataDir, "quorum-state")) - - val nodeId = if (config.processRoles.contains(ControllerRole)) { - OptionalInt.of(config.nodeId) - } else { - OptionalInt.empty() - } + val nodeId = OptionalInt.of(config.nodeId) val client = new KafkaRaftClient( recordSerde, diff --git a/core/src/main/scala/kafka/server/AclApis.scala b/core/src/main/scala/kafka/server/AclApis.scala index 97b685bc0aae2..485cafeca2038 100644 --- a/core/src/main/scala/kafka/server/AclApis.scala +++ b/core/src/main/scala/kafka/server/AclApis.scala @@ -24,14 +24,16 @@ import org.apache.kafka.common.acl.AclOperation._ import org.apache.kafka.common.acl.AclBinding import org.apache.kafka.common.errors._ import org.apache.kafka.common.message.CreateAclsResponseData.AclCreationResult +import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsFilterResult import org.apache.kafka.common.message._ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests._ import org.apache.kafka.common.resource.Resource.CLUSTER_NAME import org.apache.kafka.common.resource.ResourceType import org.apache.kafka.server.authorizer._ -import java.util +import java.util +import java.util.concurrent.CompletableFuture import scala.collection.mutable.ArrayBuffer import scala.collection.mutable import scala.compat.java8.OptionConverters._ @@ -53,7 +55,7 @@ class AclApis(authHelper: AuthHelper, def close(): Unit = alterAclsPurgatory.shutdown() - def handleDescribeAcls(request: RequestChannel.Request): Unit = { + def handleDescribeAcls(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, DESCRIBE) val describeAclsRequest = request.body[DescribeAclsRequest] authorizer match { @@ -74,9 +76,10 @@ class AclApis(authHelper: AuthHelper, .setResources(DescribeAclsResponse.aclsResources(returnedAcls)), describeAclsRequest.version)) } + CompletableFuture.completedFuture[Unit](()) } - def handleCreateAcls(request: RequestChannel.Request): Unit = { + def handleCreateAcls(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, ALTER) val createAclsRequest = request.body[CreateAclsRequest] @@ -84,6 +87,7 @@ class AclApis(authHelper: AuthHelper, case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => createAclsRequest.getErrorResponse(requestThrottleMs, new SecurityDisabledException("No Authorizer is configured."))) + CompletableFuture.completedFuture[Unit](()) case Some(auth) => val allBindings = createAclsRequest.aclCreations.asScala.map(CreateAclsRequest.aclBinding) val errorResults = mutable.Map[AclBinding, AclCreateResult]() @@ -103,6 +107,7 @@ class AclApis(authHelper: AuthHelper, validBindings += acl } + val future = new CompletableFuture[util.List[AclCreationResult]]() val createResults = auth.createAcls(request.context, validBindings.asJava).asScala.map(_.toCompletableFuture) def sendResponseCallback(): Unit = { @@ -117,17 +122,20 @@ class AclApis(authHelper: AuthHelper, } creationResult } + future.complete(aclCreationResults.asJava) + } + alterAclsPurgatory.tryCompleteElseWatch(config.connectionsMaxIdleMs, createResults, sendResponseCallback) + + future.thenApply[Unit] { aclCreationResults => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => new CreateAclsResponse(new CreateAclsResponseData() .setThrottleTimeMs(requestThrottleMs) - .setResults(aclCreationResults.asJava))) + .setResults(aclCreationResults))) } - - alterAclsPurgatory.tryCompleteElseWatch(config.connectionsMaxIdleMs, createResults, sendResponseCallback) } } - def handleDeleteAcls(request: RequestChannel.Request): Unit = { + def handleDeleteAcls(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, ALTER) val deleteAclsRequest = request.body[DeleteAclsRequest] authorizer match { @@ -135,13 +143,20 @@ class AclApis(authHelper: AuthHelper, requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => deleteAclsRequest.getErrorResponse(requestThrottleMs, new SecurityDisabledException("No Authorizer is configured."))) + CompletableFuture.completedFuture[Unit](()) case Some(auth) => + val future = new CompletableFuture[util.List[DeleteAclsFilterResult]]() val deleteResults = auth.deleteAcls(request.context, deleteAclsRequest.filters) .asScala.map(_.toCompletableFuture).toList def sendResponseCallback(): Unit = { val filterResults = deleteResults.map(_.get).map(DeleteAclsResponse.filterResult).asJava + future.complete(filterResults) + } + + alterAclsPurgatory.tryCompleteElseWatch(config.connectionsMaxIdleMs, deleteResults, sendResponseCallback) + future.thenApply[Unit] { filterResults => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => new DeleteAclsResponse( new DeleteAclsResponseData() @@ -149,7 +164,6 @@ class AclApis(authHelper: AuthHelper, .setFilterResults(filterResults), deleteAclsRequest.version)) } - alterAclsPurgatory.tryCompleteElseWatch(config.connectionsMaxIdleMs, deleteResults, sendResponseCallback) } } -} + } diff --git a/core/src/main/scala/kafka/server/ApiVersionManager.scala b/core/src/main/scala/kafka/server/ApiVersionManager.scala index 6d329673a8891..20dc043563b79 100644 --- a/core/src/main/scala/kafka/server/ApiVersionManager.scala +++ b/core/src/main/scala/kafka/server/ApiVersionManager.scala @@ -18,6 +18,7 @@ package kafka.server import kafka.network import kafka.network.RequestChannel +import org.apache.kafka.common.feature.{Features, SupportedVersionRange} import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.ApiVersionsResponse @@ -51,17 +52,18 @@ object ApiVersionManager { class SimpleApiVersionManager( val listenerType: ListenerType, - val enabledApis: collection.Set[ApiKeys] + val enabledApis: collection.Set[ApiKeys], + brokerFeatures: Features[SupportedVersionRange] ) extends ApiVersionManager { def this(listenerType: ListenerType) = { - this(listenerType, ApiKeys.apisForListener(listenerType).asScala) + this(listenerType, ApiKeys.apisForListener(listenerType).asScala, BrokerFeatures.defaultSupportedFeatures()) } private val apiVersions = ApiVersionsResponse.collectApis(enabledApis.asJava) override def apiVersionResponse(requestThrottleMs: Int): ApiVersionsResponse = { - ApiVersionsResponse.createApiVersionsResponse(0, apiVersions) + ApiVersionsResponse.createApiVersionsResponse(requestThrottleMs, apiVersions, brokerFeatures) } } diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index 70ef7c71cbba3..040529dde5768 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -71,9 +71,13 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte object BrokerFeatures extends Logging { def createDefault(): BrokerFeatures = { - new BrokerFeatures(Features.supportedFeatures( + new BrokerFeatures(defaultSupportedFeatures()) + } + + def defaultSupportedFeatures(): Features[SupportedVersionRange] = { + Features.supportedFeatures( java.util.Collections.singletonMap(MetadataVersion.FEATURE_NAME, - new SupportedVersionRange(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), MetadataVersion.latest().featureLevel())))) + new SupportedVersionRange(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), MetadataVersion.latest().featureLevel()))) } def createEmpty(): BrokerFeatures = { diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index d0d2a98b483f9..32034cb1a4f42 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -50,6 +50,7 @@ import org.apache.kafka.raft.RaftConfig.AddressSpec import org.apache.kafka.raft.{RaftClient, RaftConfig} import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.fault.FaultHandler import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.snapshot.SnapshotWriter @@ -62,13 +63,8 @@ class BrokerSnapshotWriterBuilder(raftClient: RaftClient[ApiMessageAndVersion]) extends SnapshotWriterBuilder { override def build(committedOffset: Long, committedEpoch: Int, - lastContainedLogTime: Long): SnapshotWriter[ApiMessageAndVersion] = { - raftClient.createSnapshot(committedOffset, committedEpoch, lastContainedLogTime). - asScala.getOrElse( - throw new RuntimeException("A snapshot already exists with " + - s"committedOffset=$committedOffset, committedEpoch=$committedEpoch, " + - s"lastContainedLogTime=$lastContainedLogTime") - ) + lastContainedLogTime: Long): Option[SnapshotWriter[ApiMessageAndVersion]] = { + raftClient.createSnapshot(committedOffset, committedEpoch, lastContainedLogTime).asScala } } @@ -81,12 +77,17 @@ class BrokerServer( val raftManager: RaftManager[ApiMessageAndVersion], val time: Time, val metrics: Metrics, + val brokerMetrics: BrokerServerMetrics, val threadNamePrefix: Option[String], val initialOfflineDirs: Seq[String], - val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]] + val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], + val fatalFaultHandler: FaultHandler, + val metadataLoadingFaultHandler: FaultHandler, + val metadataPublishingFaultHandler: FaultHandler ) extends KafkaBroker { - override def brokerState: BrokerState = lifecycleManager.state + override def brokerState: BrokerState = Option(lifecycleManager). + flatMap(m => Some(m.state)).getOrElse(BrokerState.NOT_RUNNING) import kafka.server.Server._ @@ -94,7 +95,7 @@ class BrokerServer( this.logIdent = logContext.logPrefix - @volatile private var lifecycleManager: BrokerLifecycleManager = null + @volatile var lifecycleManager: BrokerLifecycleManager = null private val isShuttingDown = new AtomicBoolean(false) @@ -319,8 +320,8 @@ class BrokerServer( threadNamePrefix, config.metadataSnapshotMaxNewRecordBytes, metadataSnapshotter, - BrokerServerMetrics(metrics) - ) + brokerMetrics, + metadataLoadingFaultHandler) val networkListeners = new ListenerCollection() config.effectiveAdvertisedListeners.foreach { ep => @@ -343,7 +344,7 @@ class BrokerServer( config, "heartbeat", threadNamePrefix, - config.brokerSessionTimeoutMs.toLong + config.brokerSessionTimeoutMs / 2 // KAFKA-14392 ) lifecycleManager.start( () => metadataListener.highestMetadataOffset, @@ -436,7 +437,9 @@ class BrokerServer( transactionCoordinator, clientQuotaMetadataManager, dynamicConfigHandlers.toMap, - authorizer) + authorizer, + fatalFaultHandler, + metadataPublishingFaultHandler) // Tell the metadata listener to start publishing its output, and wait for the first // publish operation to complete. This first operation will initialize logManager, diff --git a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala index a4879798342dc..92754a793f5b8 100644 --- a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala @@ -211,7 +211,7 @@ class BrokerToControllerChannelManagerImpl( 50, Selectable.USE_DEFAULT_BUFFER_SIZE, Selectable.USE_DEFAULT_BUFFER_SIZE, - config.requestTimeoutMs, + Math.min(Int.MaxValue, Math.min(config.controllerSocketTimeoutMs, retryTimeoutMs)).toInt, // request timeout should not exceed the provided retry timeout config.connectionSetupTimeoutMs, config.connectionSetupTimeoutMaxMs, time, @@ -283,7 +283,7 @@ class BrokerToControllerRequestThread( time: Time, threadName: String, retryTimeoutMs: Long -) extends InterBrokerSendThread(threadName, networkClient, config.controllerSocketTimeoutMs, time, isInterruptible = false) { +) extends InterBrokerSendThread(threadName, networkClient, Math.min(Int.MaxValue, Math.min(config.controllerSocketTimeoutMs, retryTimeoutMs)).toInt, time, isInterruptible = false) { private val requestQueue = new LinkedBlockingDeque[BrokerToControllerQueueItem]() private val activeController = new AtomicReference[Node](null) @@ -370,7 +370,7 @@ class BrokerToControllerRequestThread( debug("Controller isn't cached, looking for local metadata changes") controllerNodeProvider.get() match { case Some(controllerNode) => - info(s"Recorded new controller, from now on will use broker $controllerNode") + info(s"Recorded new controller, from now on will use node $controllerNode") updateControllerAddress(controllerNode) metadataUpdater.setNodes(Seq(controllerNode).asJava) case None => diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index ee7c70bec93c3..45c7c31570f0a 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -433,12 +433,11 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, .quota(new Quota(quotaLimit, true)) } - protected def getOrCreateSensor(sensorName: String, metricName: MetricName): Sensor = { + protected def getOrCreateSensor(sensorName: String, expirationTimeSeconds: Long, registerMetrics: Sensor => Unit): Sensor = { sensorAccessor.getOrCreate( sensorName, - ClientQuotaManager.InactiveSensorExpirationTimeSeconds, - sensor => sensor.add(metricName, new Rate) - ) + expirationTimeSeconds, + registerMetrics) } /** diff --git a/core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala b/core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala index 2ceaab9c9afdf..6e57d97bc3e5d 100644 --- a/core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala @@ -17,11 +17,11 @@ package kafka.server import java.util.concurrent.TimeUnit - import kafka.network.RequestChannel import kafka.utils.QuotaUtils import org.apache.kafka.common.MetricName import org.apache.kafka.common.metrics._ +import org.apache.kafka.common.metrics.stats.Rate import org.apache.kafka.common.utils.Time import org.apache.kafka.server.quota.ClientQuotaCallback @@ -30,6 +30,9 @@ import scala.jdk.CollectionConverters._ object ClientRequestQuotaManager { val QuotaRequestPercentDefault = Int.MaxValue.toDouble val NanosToPercentagePerSecond = 100.0 / TimeUnit.SECONDS.toNanos(1) + // Since exemptSensor is for all clients and has a constant name, we do not expire exemptSensor and only + // create once. + val DefaultInactiveExemptSensorExpirationTimeSeconds = Long.MaxValue private val ExemptSensorName = "exempt-" + QuotaType.Request } @@ -45,7 +48,9 @@ class ClientRequestQuotaManager(private val config: ClientQuotaManagerConfig, private val exemptMetricName = metrics.metricName("exempt-request-time", QuotaType.Request.toString, "Tracking exempt-request-time utilization percentage") - lazy val exemptSensor: Sensor = getOrCreateSensor(ClientRequestQuotaManager.ExemptSensorName, exemptMetricName) + val exemptSensor: Sensor = getOrCreateSensor(ClientRequestQuotaManager.ExemptSensorName, + ClientRequestQuotaManager.DefaultInactiveExemptSensorExpirationTimeSeconds, + sensor => sensor.add(exemptMetricName, new Rate)) def recordExempt(value: Double): Unit = { exemptSensor.record(value) diff --git a/core/src/main/scala/kafka/server/ConfigAdminManager.scala b/core/src/main/scala/kafka/server/ConfigAdminManager.scala index cc7a98179dd4e..c6ea4e1395381 100644 --- a/core/src/main/scala/kafka/server/ConfigAdminManager.scala +++ b/core/src/main/scala/kafka/server/ConfigAdminManager.scala @@ -497,8 +497,9 @@ object ConfigAdminManager { throw new InvalidConfigurationException(s"Config value append is not allowed for config key: ${alterConfigOp.configEntry.name}") val oldValueList = Option(configProps.getProperty(alterConfigOp.configEntry.name)) .orElse(Option(ConfigDef.convertToString(configKeys(configPropName).defaultValue, ConfigDef.Type.LIST))) - .getOrElse("") - .split(",").toList + .filter(s => s.nonEmpty) + .map(_.split(",").toList) + .getOrElse(List.empty) val appendingValueList = alterConfigOp.configEntry.value.split(",").toList.filter(value => !oldValueList.contains(value)) val newValueList = oldValueList ::: appendingValueList configProps.setProperty(alterConfigOp.configEntry.name, newValueList.mkString(",")) diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index 74bc4dd406712..1c3586263a7dd 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -20,7 +20,7 @@ package kafka.server import java.util import java.util.{Collections, OptionalLong} import java.util.Map.Entry -import java.util.concurrent.{CompletableFuture, ExecutionException} +import java.util.concurrent.CompletableFuture import kafka.network.RequestChannel import kafka.raft.RaftManager import kafka.server.QuotaFactory.QuotaManagers @@ -31,6 +31,7 @@ import org.apache.kafka.common.acl.AclOperation.{ALTER, ALTER_CONFIGS, CLUSTER_A import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors.{ApiException, ClusterAuthorizationException, InvalidRequestException, TopicDeletionDisabledException} import org.apache.kafka.common.internals.FatalExitError +import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.AlterConfigsResponseData.{AlterConfigsResourceResponse => OldAlterConfigsResourceResponse} import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult @@ -68,6 +69,7 @@ class ControllerApis(val requestChannel: RequestChannel, val controllerNodes: Seq[Node], val apiVersionManager: ApiVersionManager) extends ApiRequestHandler with Logging { + this.logIdent = s"[ControllerApis nodeId=${config.nodeId}] " val authHelper = new AuthHelper(authorizer) val requestHelper = new RequestHandlerHelper(requestChannel, quotas, time) private val aclApis = new AclApis(authHelper, authorizer, requestHelper, "controller", config) @@ -78,7 +80,7 @@ class ControllerApis(val requestChannel: RequestChannel, override def handle(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = { try { - request.header.apiKey match { + val handlerFuture: CompletableFuture[Unit] = request.header.apiKey match { case ApiKeys.FETCH => handleFetch(request) case ApiKeys.FETCH_SNAPSHOT => handleFetchSnapshot(request) case ApiKeys.CREATE_TOPICS => handleCreateTopics(request) @@ -109,10 +111,21 @@ class ControllerApis(val requestChannel: RequestChannel, case ApiKeys.UPDATE_FEATURES => handleUpdateFeatures(request) case _ => throw new ApiException(s"Unsupported ApiKey ${request.context.header.apiKey}") } + + // This catches exceptions in the future and subsequent completion stages returned by the request handlers. + handlerFuture.whenComplete { (_, exception) => + if (exception != null) { + // CompletionException does not include the stack frames in its "cause" exception, so we need to + // log the original exception here + error(s"Unexpected error handling request ${request.requestDesc(true)} " + + s"with context ${request.context}", exception) + requestHelper.handleError(request, exception) + } + } } catch { case e: FatalExitError => throw e - case e: Throwable => { - val t = if (e.isInstanceOf[ExecutionException]) e.getCause else e + case t: Throwable => { + // This catches exceptions in the blocking parts of the request handlers error(s"Unexpected error handling request ${request.requestDesc(true)} " + s"with context ${request.context}", t) requestHelper.handleError(request, t) @@ -125,38 +138,41 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleEnvelopeRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = { + def handleEnvelopeRequest(request: RequestChannel.Request, requestLocal: RequestLocal): CompletableFuture[Unit] = { if (!authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) { requestHelper.sendErrorResponseMaybeThrottle(request, new ClusterAuthorizationException( s"Principal ${request.context.principal} does not have required CLUSTER_ACTION for envelope")) } else { EnvelopeUtils.handleEnvelopeRequest(request, requestChannel.metrics, handle(_, requestLocal)) } + CompletableFuture.completedFuture[Unit](()) } - def handleSaslHandshakeRequest(request: RequestChannel.Request): Unit = { + def handleSaslHandshakeRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { val responseData = new SaslHandshakeResponseData().setErrorCode(ILLEGAL_SASL_STATE.code) requestHelper.sendResponseMaybeThrottle(request, _ => new SaslHandshakeResponse(responseData)) + CompletableFuture.completedFuture[Unit](()) } - def handleSaslAuthenticateRequest(request: RequestChannel.Request): Unit = { + def handleSaslAuthenticateRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { val responseData = new SaslAuthenticateResponseData() .setErrorCode(ILLEGAL_SASL_STATE.code) .setErrorMessage("SaslAuthenticate request received after successful authentication") requestHelper.sendResponseMaybeThrottle(request, _ => new SaslAuthenticateResponse(responseData)) + CompletableFuture.completedFuture[Unit](()) } - def handleFetch(request: RequestChannel.Request): Unit = { + def handleFetch(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new FetchResponse(response.asInstanceOf[FetchResponseData])) } - def handleFetchSnapshot(request: RequestChannel.Request): Unit = { + def handleFetchSnapshot(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new FetchSnapshotResponse(response.asInstanceOf[FetchSnapshotResponseData])) } - def handleDeleteTopics(request: RequestChannel.Request): Unit = { + def handleDeleteTopics(request: RequestChannel.Request): CompletableFuture[Unit] = { val deleteTopicsRequest = request.body[DeleteTopicsRequest] val context = new ControllerRequestContext(request.context.header.data, request.context.principal, requestTimeoutMsToDeadlineNs(time, deleteTopicsRequest.data.timeoutMs)) @@ -166,7 +182,7 @@ class ControllerApis(val requestChannel: RequestChannel, authHelper.authorize(request.context, DELETE, CLUSTER, CLUSTER_NAME, logIfDenied = false), names => authHelper.filterByAuthorized(request.context, DESCRIBE, TOPIC, names)(n => n), names => authHelper.filterByAuthorized(request.context, DELETE, TOPIC, names)(n => n)) - future.whenComplete { (results, exception) => + future.handle[Unit] { (results, exception) => requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => { if (exception != null) { deleteTopicsRequest.getErrorResponse(throttleTimeMs, exception) @@ -320,7 +336,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleCreateTopics(request: RequestChannel.Request): Unit = { + def handleCreateTopics(request: RequestChannel.Request): CompletableFuture[Unit] = { val createTopicsRequest = request.body[CreateTopicsRequest] val context = new ControllerRequestContext(request.context.header.data, request.context.principal, requestTimeoutMsToDeadlineNs(time, createTopicsRequest.data.timeoutMs)) @@ -330,7 +346,7 @@ class ControllerApis(val requestChannel: RequestChannel, names => authHelper.filterByAuthorized(request.context, CREATE, TOPIC, names)(identity), names => authHelper.filterByAuthorized(request.context, DESCRIBE_CONFIGS, TOPIC, names, logIfDenied = false)(identity)) - future.whenComplete { (result, exception) => + future.handle[Unit] { (result, exception) => requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => { if (exception != null) { createTopicsRequest.getErrorResponse(throttleTimeMs, exception) @@ -359,12 +375,21 @@ class ControllerApis(val requestChannel: RequestChannel, } } } + + /* The cluster metatdata topic is an internal topic with a different implementation. The user should not be + * allowed to create it as a regular topic. + */ + if (topicNames.contains(Topic.CLUSTER_METADATA_TOPIC_NAME)) { + info(s"Rejecting creation of internal topic ${Topic.CLUSTER_METADATA_TOPIC_NAME}") + } + val allowedTopicNames = topicNames.asScala.diff(Set(Topic.CLUSTER_METADATA_TOPIC_NAME)) + val authorizedTopicNames = if (hasClusterAuth) { - topicNames.asScala + allowedTopicNames } else { - getCreatableTopics.apply(topicNames.asScala) + getCreatableTopics.apply(allowedTopicNames) } - val describableTopicNames = getDescribableTopics.apply(topicNames.asScala).asJava + val describableTopicNames = getDescribableTopics.apply(allowedTopicNames).asJava val effectiveRequest = request.duplicate() val iterator = effectiveRequest.topics().iterator() while (iterator.hasNext) { @@ -385,14 +410,15 @@ class ControllerApis(val requestChannel: RequestChannel, if (!authorizedTopicNames.contains(name)) { response.topics().add(new CreatableTopicResult(). setName(name). - setErrorCode(TOPIC_AUTHORIZATION_FAILED.code)) + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code). + setErrorMessage("Authorization failed.")) } } response } } - def handleApiVersionsRequest(request: RequestChannel.Request): Unit = { + def handleApiVersionsRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { // Note that broker returns its full list of supported ApiKeys and versions regardless of current // authentication state (e.g., before SASL authentication on an SASL listener, do note that no // Kafka protocol requests may take place on an SSL listener before the SSL handshake is finished). @@ -410,6 +436,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } requestHelper.sendResponseMaybeThrottle(request, createResponseCallback) + CompletableFuture.completedFuture[Unit](()) } def authorizeAlterResource(requestContext: RequestContext, @@ -431,7 +458,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleLegacyAlterConfigs(request: RequestChannel.Request): Unit = { + def handleLegacyAlterConfigs(request: RequestChannel.Request): CompletableFuture[Unit] = { val response = new AlterConfigsResponseData() val alterConfigsRequest = request.body[AlterConfigsRequest] val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) @@ -474,7 +501,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } controller.legacyAlterConfigs(context, configChanges, alterConfigsRequest.data.validateOnly) - .whenComplete { (controllerResults, exception) => + .handle[Unit] { (controllerResults, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { @@ -490,33 +517,33 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleVote(request: RequestChannel.Request): Unit = { + def handleVote(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new VoteResponse(response.asInstanceOf[VoteResponseData])) } - def handleBeginQuorumEpoch(request: RequestChannel.Request): Unit = { + def handleBeginQuorumEpoch(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new BeginQuorumEpochResponse(response.asInstanceOf[BeginQuorumEpochResponseData])) } - def handleEndQuorumEpoch(request: RequestChannel.Request): Unit = { + def handleEndQuorumEpoch(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new EndQuorumEpochResponse(response.asInstanceOf[EndQuorumEpochResponseData])) } - def handleDescribeQuorum(request: RequestChannel.Request): Unit = { + def handleDescribeQuorum(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, DESCRIBE) handleRaftRequest(request, response => new DescribeQuorumResponse(response.asInstanceOf[DescribeQuorumResponseData])) } - def handleElectLeaders(request: RequestChannel.Request): Unit = { + def handleElectLeaders(request: RequestChannel.Request): CompletableFuture[Unit] = { authHelper.authorizeClusterOperation(request, ALTER) val electLeadersRequest = request.body[ElectLeadersRequest] val context = new ControllerRequestContext(request.context.header.data, request.context.principal, requestTimeoutMsToDeadlineNs(time, electLeadersRequest.data.timeoutMs)) val future = controller.electLeaders(context, electLeadersRequest.data) - future.whenComplete { (responseData, exception) => + future.handle[Unit] { (responseData, exception) => if (exception != null) { requestHelper.sendResponseMaybeThrottle(request, throttleMs => { electLeadersRequest.getErrorResponse(throttleMs, exception) @@ -529,13 +556,13 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleAlterPartitionRequest(request: RequestChannel.Request): Unit = { + def handleAlterPartitionRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { val alterPartitionRequest = request.body[AlterPartitionRequest] val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) val future = controller.alterPartition(context, alterPartitionRequest.data) - future.whenComplete { (result, exception) => + future.handle[Unit] { (result, exception) => val response = if (exception != null) { alterPartitionRequest.getErrorResponse(exception) } else { @@ -545,11 +572,11 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleBrokerHeartBeatRequest(request: RequestChannel.Request): Unit = { + def handleBrokerHeartBeatRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { val heartbeatRequest = request.body[BrokerHeartbeatRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, - requestTimeoutMsToDeadlineNs(time, config.brokerHeartbeatIntervalMs)) + requestTimeoutMsToDeadlineNs(time, config.brokerHeartbeatIntervalMs / 2)) controller.processBrokerHeartbeat(context, heartbeatRequest.data).handle[Unit] { (reply, e) => def createResponseCallback(requestThrottleMs: Int, reply: BrokerHeartbeatReply, @@ -572,7 +599,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleUnregisterBroker(request: RequestChannel.Request): Unit = { + def handleUnregisterBroker(request: RequestChannel.Request): CompletableFuture[Unit] = { val decommissionRequest = request.body[UnregisterBrokerRequest] authHelper.authorizeClusterOperation(request, ALTER) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, @@ -595,7 +622,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleBrokerRegistration(request: RequestChannel.Request): Unit = { + def handleBrokerRegistration(request: RequestChannel.Request): CompletableFuture[Unit] = { val registrationRequest = request.body[BrokerRegistrationRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, @@ -622,11 +649,10 @@ class ControllerApis(val requestChannel: RequestChannel, } private def handleRaftRequest(request: RequestChannel.Request, - buildResponse: ApiMessage => AbstractResponse): Unit = { + buildResponse: ApiMessage => AbstractResponse): CompletableFuture[Unit] = { val requestBody = request.body[AbstractRequest] val future = raftManager.handleRequest(request.header, requestBody.data, time.milliseconds()) - - future.whenComplete { (responseData, exception) => + future.handle[Unit] { (responseData, exception) => val response = if (exception != null) { requestBody.getErrorResponse(exception) } else { @@ -636,13 +662,13 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleAlterClientQuotas(request: RequestChannel.Request): Unit = { + def handleAlterClientQuotas(request: RequestChannel.Request): CompletableFuture[Unit] = { val quotaRequest = request.body[AlterClientQuotasRequest] authHelper.authorizeClusterOperation(request, ALTER_CONFIGS) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) controller.alterClientQuotas(context, quotaRequest.entries, quotaRequest.validateOnly) - .whenComplete { (results, exception) => + .handle[Unit] { (results, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { @@ -652,7 +678,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleIncrementalAlterConfigs(request: RequestChannel.Request): Unit = { + def handleIncrementalAlterConfigs(request: RequestChannel.Request): CompletableFuture[Unit] = { val response = new IncrementalAlterConfigsResponseData() val alterConfigsRequest = request.body[IncrementalAlterConfigsRequest] val context = new ControllerRequestContext(request.context.header.data, request.context.principal, @@ -700,7 +726,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } controller.incrementalAlterConfigs(context, configChanges, alterConfigsRequest.data.validateOnly) - .whenComplete { (controllerResults, exception) => + .handle[Unit] { (controllerResults, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { @@ -716,7 +742,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleCreatePartitions(request: RequestChannel.Request): Unit = { + def handleCreatePartitions(request: RequestChannel.Request): CompletableFuture[Unit] = { def filterAlterAuthorizedTopics(topics: Iterable[String]): Set[String] = { authHelper.filterByAuthorized(request.context, ALTER, TOPIC, topics)(n => n) } @@ -726,7 +752,7 @@ class ControllerApis(val requestChannel: RequestChannel, val future = createPartitions(context, createPartitionsRequest.data(), filterAlterAuthorizedTopics) - future.whenComplete { (responses, exception) => + future.handle[Unit] { (responses, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { @@ -778,33 +804,37 @@ class ControllerApis(val requestChannel: RequestChannel, } } - def handleAlterPartitionReassignments(request: RequestChannel.Request): Unit = { + def handleAlterPartitionReassignments(request: RequestChannel.Request): CompletableFuture[Unit] = { val alterRequest = request.body[AlterPartitionReassignmentsRequest] authHelper.authorizeClusterOperation(request, ALTER) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, requestTimeoutMsToDeadlineNs(time, alterRequest.data.timeoutMs)) - val response = controller.alterPartitionReassignments(context, alterRequest.data).get() - requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - new AlterPartitionReassignmentsResponse(response.setThrottleTimeMs(requestThrottleMs))) + controller.alterPartitionReassignments(context, alterRequest.data) + .thenApply[Unit] { response => + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + new AlterPartitionReassignmentsResponse(response.setThrottleTimeMs(requestThrottleMs))) + } } - def handleListPartitionReassignments(request: RequestChannel.Request): Unit = { + def handleListPartitionReassignments(request: RequestChannel.Request): CompletableFuture[Unit] = { val listRequest = request.body[ListPartitionReassignmentsRequest] authHelper.authorizeClusterOperation(request, DESCRIBE) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) - val response = controller.listPartitionReassignments(context, listRequest.data).get() - requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - new ListPartitionReassignmentsResponse(response.setThrottleTimeMs(requestThrottleMs))) + controller.listPartitionReassignments(context, listRequest.data) + .thenApply[Unit] { response => + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + new ListPartitionReassignmentsResponse(response.setThrottleTimeMs(requestThrottleMs))) + } } - def handleAllocateProducerIdsRequest(request: RequestChannel.Request): Unit = { + def handleAllocateProducerIdsRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { val allocatedProducerIdsRequest = request.body[AllocateProducerIdsRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) controller.allocateProducerIds(context, allocatedProducerIdsRequest.data) - .whenComplete((results, exception) => { + .handle[Unit] { (results, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { @@ -813,22 +843,22 @@ class ControllerApis(val requestChannel: RequestChannel, new AllocateProducerIdsResponse(results) }) } - }) + } } - def handleUpdateFeatures(request: RequestChannel.Request): Unit = { + def handleUpdateFeatures(request: RequestChannel.Request): CompletableFuture[Unit] = { val updateFeaturesRequest = request.body[UpdateFeaturesRequest] authHelper.authorizeClusterOperation(request, ALTER) val context = new ControllerRequestContext(request.context.header.data, request.context.principal, OptionalLong.empty()) controller.updateFeatures(context, updateFeaturesRequest.data) - .whenComplete((response, exception) => { + .handle[Unit] { (response, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => new UpdateFeaturesResponse(response.setThrottleTimeMs(requestThrottleMs))) } - }) + } } } diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 67b3f0276d7a0..8e52f30d2ea7f 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -21,13 +21,13 @@ import java.util import java.util.OptionalLong import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.{CompletableFuture, TimeUnit} - import kafka.cluster.Broker.ServerInfo import kafka.metrics.{KafkaMetricsGroup, LinuxIoMetricsCollector} import kafka.network.{DataPlaneAcceptor, SocketServer} import kafka.raft.RaftManager import kafka.security.CredentialProvider import kafka.server.KafkaConfig.{AlterConfigPolicyClassNameProp, CreateTopicPolicyClassNameProp} +import kafka.server.KafkaRaftServer.BrokerRole import kafka.server.QuotaFactory.QuotaManagers import kafka.utils.{CoreUtils, Logging} import org.apache.kafka.clients.ApiVersions @@ -37,7 +37,7 @@ import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.{ClusterResource, Endpoint} -import org.apache.kafka.controller.{BootstrapMetadata, Controller, QuorumController, QuorumControllerMetrics, QuorumFeatures} +import org.apache.kafka.controller.{Controller, ControllerMetrics, QuorumController, QuorumFeatures} import org.apache.kafka.metadata.KafkaConfigSchema import org.apache.kafka.raft.RaftConfig import org.apache.kafka.raft.RaftConfig.AddressSpec @@ -45,6 +45,8 @@ import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.common.config.ConfigException import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer +import org.apache.kafka.metadata.bootstrap.BootstrapMetadata +import org.apache.kafka.server.fault.FaultHandler import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.policy.{AlterConfigPolicy, CreateTopicPolicy} @@ -60,14 +62,19 @@ class ControllerServer( val raftManager: RaftManager[ApiMessageAndVersion], val time: Time, val metrics: Metrics, + val controllerMetrics: ControllerMetrics, val threadNamePrefix: Option[String], val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], val configSchema: KafkaConfigSchema, val raftApiVersions: ApiVersions, - val bootstrapMetadata: BootstrapMetadata + val bootstrapMetadata: BootstrapMetadata, + val metadataFaultHandler: FaultHandler, + val fatalFaultHandler: FaultHandler, ) extends Logging with KafkaMetricsGroup { import kafka.server.Server._ + config.dynamicConfig.initialize(zkClientOpt = None) + val lock = new ReentrantLock() val awaitShutdownCond = lock.newCondition() var status: ProcessStatus = SHUTDOWN @@ -97,6 +104,13 @@ class ControllerServer( true } + private def doRemoteKraftSetup(): Unit = { + // Explicitly configure metric reporters on this remote controller. + // We do not yet support dynamic reconfiguration on remote controllers in general; + // remove this once that is implemented. + new DynamicMetricReporterState(config.nodeId, config, metrics, clusterId) + } + def clusterId: String = metaProperties.clusterId def startup(): Unit = { @@ -189,12 +203,13 @@ class ControllerServer( setSnapshotMaxNewRecordBytes(config.metadataSnapshotMaxNewRecordBytes). setLeaderImbalanceCheckIntervalNs(leaderImbalanceCheckIntervalNs). setMaxIdleIntervalNs(maxIdleIntervalNs). - setMetrics(new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry(), time)). + setMetrics(controllerMetrics). setCreateTopicPolicy(createTopicPolicy.asJava). setAlterConfigPolicy(alterConfigPolicy.asJava). setConfigurationValidator(new ControllerConfigurationValidator()). setStaticConfig(config.originals). - setBootstrapMetadata(bootstrapMetadata) + setBootstrapMetadata(bootstrapMetadata). + setFatalFaultHandler(fatalFaultHandler) } authorizer match { case Some(a: ClusterMetadataAuthorizer) => controllerBuilder.setAuthorizer(a) @@ -202,6 +217,11 @@ class ControllerServer( } controller = controllerBuilder.build() + // Perform any setup that is done only when this node is a controller-only node. + if (!config.processRoles.contains(BrokerRole)) { + doRemoteKraftSetup() + } + quotaManagers = QuotaFactory.instantiate(config, metrics, time, threadNamePrefix.getOrElse("")) controllerApis = new ControllerApis(socketServer.dataPlaneRequestChannel, authorizer, diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 33511147e6b4c..a860938124e6a 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -30,7 +30,7 @@ import kafka.utils.Implicits._ import kafka.zk.{AdminZkClient, KafkaZkClient} import org.apache.kafka.common.Reconfigurable import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, SslConfigs} -import org.apache.kafka.common.metrics.MetricsReporter +import org.apache.kafka.common.metrics.{Metrics, MetricsReporter} import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.network.{ListenerName, ListenerReconfigurable} import org.apache.kafka.common.security.authenticator.LoginManager @@ -211,7 +211,11 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging private val brokerReconfigurables = new CopyOnWriteArrayList[BrokerReconfigurable]() private val lock = new ReentrantReadWriteLock private var currentConfig: KafkaConfig = null - private val dynamicConfigPasswordEncoder = maybeCreatePasswordEncoder(kafkaConfig.passwordEncoderSecret) + private val dynamicConfigPasswordEncoder = if (kafkaConfig.processRoles.isEmpty) { + maybeCreatePasswordEncoder(kafkaConfig.passwordEncoderSecret) + } else { + Some(PasswordEncoder.noop()) + } private[server] def initialize(zkClientOpt: Option[KafkaZkClient]): Unit = { currentConfig = new KafkaConfig(kafkaConfig.props, false, None) @@ -254,7 +258,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging case _ => } addReconfigurable(kafkaServer.kafkaYammerMetrics) - addReconfigurable(new DynamicMetricsReporters(kafkaConfig.brokerId, kafkaServer)) + addReconfigurable(new DynamicMetricsReporters(kafkaConfig.brokerId, kafkaServer.config, kafkaServer.metrics, kafkaServer.clusterId)) addReconfigurable(new DynamicClientQuotaCallback(kafkaServer)) addBrokerReconfigurable(new DynamicThreadPool(kafkaServer)) @@ -340,7 +344,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging private def maybeCreatePasswordEncoder(secret: Option[Password]): Option[PasswordEncoder] = { secret.map { secret => - new PasswordEncoder(secret, + PasswordEncoder.encrypting(secret, kafkaConfig.passwordEncoderKeyFactoryAlgorithm, kafkaConfig.passwordEncoderCipherAlgorithm, kafkaConfig.passwordEncoderKeyLength, @@ -744,17 +748,18 @@ class DynamicThreadPool(server: KafkaBroker) extends BrokerReconfigurable { } } -class DynamicMetricsReporters(brokerId: Int, server: KafkaBroker) extends Reconfigurable { +class DynamicMetricsReporters(brokerId: Int, config: KafkaConfig, metrics: Metrics, clusterId: String) extends Reconfigurable { + private val reporterState = new DynamicMetricReporterState(brokerId, config, metrics, clusterId) + private val currentReporters = reporterState.currentReporters + private val dynamicConfig = reporterState.dynamicConfig - private val dynamicConfig = server.config.dynamicConfig - private val metrics = server.metrics - private val propsOverride = Map[String, AnyRef](KafkaConfig.BrokerIdProp -> brokerId.toString) - private val currentReporters = mutable.Map[String, MetricsReporter]() + private def metricsReporterClasses(configs: util.Map[String, _]): mutable.Buffer[String] = + reporterState.metricsReporterClasses(configs) - createReporters(dynamicConfig.currentKafkaConfig.getList(KafkaConfig.MetricReporterClassesProp), - Collections.emptyMap[String, Object]) + private def createReporters(reporterClasses: util.List[String], updatedConfigs: util.Map[String, _]): Unit = + reporterState.createReporters(reporterClasses, updatedConfigs) - private[server] def currentMetricsReporters: List[MetricsReporter] = currentReporters.values.toList + private def removeReporter(className: String): Unit = reporterState.removeReporter(className) override def configure(configs: util.Map[String, _]): Unit = {} @@ -797,8 +802,23 @@ class DynamicMetricsReporters(brokerId: Int, server: KafkaBroker) extends Reconf val added = updatedMetricsReporters.filterNot(currentReporters.keySet) createReporters(added.asJava, configs) } +} + +class DynamicMetricReporterState(brokerId: Int, config: KafkaConfig, metrics: Metrics, clusterId: String) { + private[server] val dynamicConfig = config.dynamicConfig + private val propsOverride = Map[String, AnyRef](KafkaConfig.BrokerIdProp -> brokerId.toString) + private[server] val currentReporters = mutable.Map[String, MetricsReporter]() + createReporters(config, clusterId, metricsReporterClasses(dynamicConfig.currentKafkaConfig.values()).asJava, + Collections.emptyMap[String, Object]) + + private[server] def createReporters(reporterClasses: util.List[String], + updatedConfigs: util.Map[String, _]): Unit = { + createReporters(config, clusterId, reporterClasses, updatedConfigs) + } - private def createReporters(reporterClasses: util.List[String], + private def createReporters(config: KafkaConfig, + clusterId: String, + reporterClasses: util.List[String], updatedConfigs: util.Map[String, _]): Unit = { val props = new util.HashMap[String, AnyRef] updatedConfigs.forEach((k, v) => props.put(k, v.asInstanceOf[AnyRef])) @@ -807,19 +827,19 @@ class DynamicMetricsReporters(brokerId: Int, server: KafkaBroker) extends Reconf // Call notifyMetricsReporters first to satisfy the contract for MetricsReporter.contextChange, // which provides that MetricsReporter.contextChange must be called before the first call to MetricsReporter.init. // The first call to MetricsReporter.init is done when we call metrics.addReporter below. - KafkaBroker.notifyMetricsReporters(server.clusterId, server.config, reporters.asScala) + KafkaBroker.notifyMetricsReporters(clusterId, config, reporters.asScala) reporters.forEach { reporter => metrics.addReporter(reporter) currentReporters += reporter.getClass.getName -> reporter } - KafkaBroker.notifyClusterListeners(server.clusterId, reporters.asScala) + KafkaBroker.notifyClusterListeners(clusterId, reporters.asScala) } - private def removeReporter(className: String): Unit = { + private[server] def removeReporter(className: String): Unit = { currentReporters.remove(className).foreach(metrics.removeReporter) } - private def metricsReporterClasses(configs: util.Map[String, _]): mutable.Buffer[String] = { + private[server] def metricsReporterClasses(configs: util.Map[String, _]): mutable.Buffer[String] = { configs.get(KafkaConfig.MetricReporterClassesProp).asInstanceOf[util.List[String]].asScala } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 4703118ebfa24..07e33112bcf4c 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -246,6 +246,7 @@ class KafkaApis(val requestChannel: RequestChannel, // try to complete delayed action. In order to avoid conflicting locking, the actions to complete delayed requests // are kept in a queue. We add the logic to check the ReplicaManager queue at the end of KafkaApis.handle() and the // expiration thread for certain delayed operations (e.g. DelayedJoin) + // Delayed fetches are also completed by ReplicaFetcherThread. replicaManager.tryCompleteActions() // The local completion time may be set while processing the request. Only record it if it's unset. if (request.apiLocalCompleteTimeNanos < 0) @@ -1924,12 +1925,36 @@ class KafkaApis(val requestChannel: RequestChannel, } val hasClusterAuthorization = authHelper.authorize(request.context, CREATE, CLUSTER, CLUSTER_NAME, logIfDenied = false) - val topics = createTopicsRequest.data.topics.asScala.map(_.name) - val authorizedTopics = - if (hasClusterAuthorization) topics.toSet - else authHelper.filterByAuthorized(request.context, CREATE, TOPIC, topics)(identity) - val authorizedForDescribeConfigs = authHelper.filterByAuthorized(request.context, DESCRIBE_CONFIGS, TOPIC, - topics, logIfDenied = false)(identity).map(name => name -> results.find(name)).toMap + + val allowedTopicNames = { + val topicNames = createTopicsRequest + .data + .topics + .asScala + .map(_.name) + .toSet + + /* The cluster metatdata topic is an internal topic with a different implementation. The user should not be + * allowed to create it as a regular topic. + */ + if (topicNames.contains(Topic.CLUSTER_METADATA_TOPIC_NAME)) { + info(s"Rejecting creation of internal topic ${Topic.CLUSTER_METADATA_TOPIC_NAME}") + } + topicNames.diff(Set(Topic.CLUSTER_METADATA_TOPIC_NAME)) + } + + val authorizedTopics = if (hasClusterAuthorization) { + allowedTopicNames.toSet + } else { + authHelper.filterByAuthorized(request.context, CREATE, TOPIC, allowedTopicNames)(identity) + } + val authorizedForDescribeConfigs = authHelper.filterByAuthorized( + request.context, + DESCRIBE_CONFIGS, + TOPIC, + allowedTopicNames, + logIfDenied = false + )(identity).map(name => name -> results.find(name)).toMap results.forEach { topic => if (results.findAll(topic.name).size > 1) { diff --git a/core/src/main/scala/kafka/server/KafkaBroker.scala b/core/src/main/scala/kafka/server/KafkaBroker.scala index 46f2e7e8b19c7..b02b1167c50be 100644 --- a/core/src/main/scala/kafka/server/KafkaBroker.scala +++ b/core/src/main/scala/kafka/server/KafkaBroker.scala @@ -89,6 +89,7 @@ trait KafkaBroker extends KafkaMetricsGroup { def shutdown(): Unit def brokerTopicStats: BrokerTopicStats def credentialProvider: CredentialProvider + def clientToControllerChannelManager: BrokerToControllerChannelManager // For backwards compatibility, we need to keep older metrics tied // to their original name when this class was named `KafkaServer` diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index a9fbda6c21079..ca7c474fdcff4 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -81,7 +81,7 @@ object Defaults { val BrokerHeartbeatIntervalMs = 2000 val BrokerSessionTimeoutMs = 9000 val MetadataSnapshotMaxNewRecordBytes = 20 * 1024 * 1024 - val MetadataMaxIdleIntervalMs = 5000 + val MetadataMaxIdleIntervalMs = 500 /** KRaft mode configs */ val EmptyNodeId: Int = -1 @@ -258,7 +258,8 @@ object Defaults { /** ********* General Security configuration ***********/ val ConnectionsMaxReauthMsDefault = 0L - val DefaultPrincipalSerde = classOf[DefaultKafkaPrincipalBuilder] + val DefaultServerMaxMaxReceiveSize = BrokerSecurityConfigs.DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE + val DefaultPrincipalBuilder = classOf[DefaultKafkaPrincipalBuilder] /** ********* Sasl configuration ***********/ val SaslMechanismInterBrokerProtocol = SaslConfigs.DEFAULT_SASL_MECHANISM @@ -565,6 +566,7 @@ object KafkaConfig { /** ******** Common Security Configuration *************/ val PrincipalBuilderClassProp = BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG val ConnectionsMaxReauthMsProp = BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS + val SaslServerMaxReceiveSizeProp = BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG val securityProviderClassProp = SecurityConfig.SECURITY_PROVIDERS_CONFIG /** ********* SSL Configuration ****************/ @@ -710,6 +712,7 @@ object KafkaConfig { val BrokerHeartbeatIntervalMsDoc = "The length of time in milliseconds between broker heartbeats. Used when running in KRaft mode." val BrokerSessionTimeoutMsDoc = "The length of time in milliseconds that a broker lease lasts if no heartbeats are made. Used when running in KRaft mode." val NodeIdDoc = "The node ID associated with the roles this process is playing when `process.roles` is non-empty. " + + "Every node in a KRaft cluster must have a unique `node.id`, this includes broker and controller nodes. " + "This is required configuration when running in KRaft mode." val MetadataLogDirDoc = "This configuration determines where we put the metadata log for clusters in KRaft mode. " + "If it is not set, the metadata log is placed in the first log directory from log.dirs." @@ -1009,6 +1012,7 @@ object KafkaConfig { /** ******** Common Security Configuration *************/ val PrincipalBuilderClassDoc = BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DOC val ConnectionsMaxReauthMsDoc = BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_DOC + val SaslServerMaxReceiveSizeDoc = BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_DOC val securityProviderClassDoc = SecurityConfig.SECURITY_PROVIDERS_DOC /** ********* SSL Configuration ****************/ @@ -1315,10 +1319,11 @@ object KafkaConfig { /** ********* General Security Configuration ****************/ .define(ConnectionsMaxReauthMsProp, LONG, Defaults.ConnectionsMaxReauthMsDefault, MEDIUM, ConnectionsMaxReauthMsDoc) + .define(SaslServerMaxReceiveSizeProp, INT, Defaults.DefaultServerMaxMaxReceiveSize, MEDIUM, SaslServerMaxReceiveSizeDoc) .define(securityProviderClassProp, STRING, null, LOW, securityProviderClassDoc) /** ********* SSL Configuration ****************/ - .define(PrincipalBuilderClassProp, CLASS, Defaults.DefaultPrincipalSerde, MEDIUM, PrincipalBuilderClassDoc) + .define(PrincipalBuilderClassProp, CLASS, Defaults.DefaultPrincipalBuilder, MEDIUM, PrincipalBuilderClassDoc) .define(SslProtocolProp, STRING, Defaults.SslProtocol, MEDIUM, SslProtocolDoc) .define(SslProviderProp, STRING, null, MEDIUM, SslProviderDoc) .define(SslEnabledProtocolsProp, LIST, Defaults.SslEnabledProtocols, MEDIUM, SslEnabledProtocolsDoc) @@ -1493,6 +1498,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami // Cache the current config to avoid acquiring read lock to access from dynamicConfig @volatile private var currentConfig = this + val processRoles: Set[ProcessRole] = parseProcessRoles() private[server] val dynamicConfig = dynamicConfigOverride.getOrElse(new DynamicBrokerConfig(this)) private[server] def updateCurrentConfig(newConfig: KafkaConfig): Unit = { @@ -1612,7 +1618,6 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val maxReservedBrokerId: Int = getInt(KafkaConfig.MaxReservedBrokerIdProp) var brokerId: Int = getInt(KafkaConfig.BrokerIdProp) val nodeId: Int = getInt(KafkaConfig.NodeIdProp) - val processRoles: Set[ProcessRole] = parseProcessRoles() val initialRegistrationTimeoutMs: Int = getInt(KafkaConfig.InitialBrokerRegistrationTimeoutMsProp) val brokerHeartbeatIntervalMs: Int = getInt(KafkaConfig.BrokerHeartbeatIntervalMsProp) val brokerSessionTimeoutMs: Int = getInt(KafkaConfig.BrokerSessionTimeoutMsProp) @@ -2157,14 +2162,16 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami // KRaft controller-only validateNonEmptyQuorumVotersForKRaft() validateControlPlaneListenerEmptyForKRaft() - // advertised listeners must be empty when not also running the broker role - val sourceOfAdvertisedListeners: String = - if (getString(KafkaConfig.AdvertisedListenersProp) != null) - s"${KafkaConfig.AdvertisedListenersProp}" - else - s"${KafkaConfig.ListenersProp}" - require(effectiveAdvertisedListeners.isEmpty, - s"The $sourceOfAdvertisedListeners config must only contain KRaft controller listeners from ${KafkaConfig.ControllerListenerNamesProp} when ${KafkaConfig.ProcessRolesProp}=controller") + // advertised listeners must be empty when only the controller is configured + require( + getString(KafkaConfig.AdvertisedListenersProp) == null, + s"The ${KafkaConfig.AdvertisedListenersProp} config must be empty when ${KafkaConfig.ProcessRolesProp}=controller" + ) + // listeners should only contain listeners also enumerated in the controller listener + require( + effectiveAdvertisedListeners.isEmpty, + s"The ${KafkaConfig.ListenersProp} config must only contain KRaft controller listeners from ${KafkaConfig.ControllerListenerNamesProp} when ${KafkaConfig.ProcessRolesProp}=controller" + ) validateControllerQuorumVotersMustContainNodeIdForKRaftController() validateControllerListenerExistsForKRaftController() validateControllerListenerNamesMustAppearInListenersForKRaftController() diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index 07a3118372046..76a874b2197ff 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -23,20 +23,22 @@ import kafka.log.{LogConfig, UnifiedLog} import kafka.metrics.KafkaMetricsReporter import kafka.raft.KafkaRaftManager import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole} +import kafka.server.metadata.BrokerServerMetrics import kafka.utils.{CoreUtils, Logging, Mx4jLoader, VerifiableProperties} import org.apache.kafka.common.config.{ConfigDef, ConfigResource} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.utils.{AppInfoParser, Time} import org.apache.kafka.common.{KafkaException, Uuid} -import org.apache.kafka.controller.BootstrapMetadata +import org.apache.kafka.controller.QuorumControllerMetrics +import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata} import org.apache.kafka.metadata.{KafkaConfigSchema, MetadataRecordSerde} import org.apache.kafka.raft.RaftConfig -import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} +import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.fault.{LoggingFaultHandler, ProcessExitingFaultHandler} import org.apache.kafka.server.metrics.KafkaYammerMetrics -import java.nio.file.Paths +import java.util.Optional import scala.collection.Seq -import scala.compat.java8.FunctionConverters.asJavaSupplier import scala.jdk.CollectionConverters._ /** @@ -45,8 +47,6 @@ import scala.jdk.CollectionConverters._ * constructing the controller and/or broker based on the `process.roles` * configuration and for managing their basic lifecycle (startup and shutdown). * - * Note that this server is a work in progress and we are releasing it as - * early access in 2.8.0. */ class KafkaRaftServer( config: KafkaConfig, @@ -54,6 +54,7 @@ class KafkaRaftServer( threadNamePrefix: Option[String] ) extends Server with Logging { + this.logIdent = s"[KafkaRaftServer nodeId=${config.nodeId}] " KafkaMetricsReporter.startReporters(VerifiableProperties(config.originals)) KafkaYammerMetrics.INSTANCE.configure(config.originals) @@ -81,32 +82,49 @@ class KafkaRaftServer( ) private val broker: Option[BrokerServer] = if (config.processRoles.contains(BrokerRole)) { + val brokerMetrics = BrokerServerMetrics(metrics) + val fatalFaultHandler = new ProcessExitingFaultHandler() + val metadataLoadingFaultHandler = new LoggingFaultHandler("metadata loading", + () => brokerMetrics.metadataLoadErrorCount.getAndIncrement()) + val metadataApplyingFaultHandler = new LoggingFaultHandler("metadata application", + () => brokerMetrics.metadataApplyErrorCount.getAndIncrement()) Some(new BrokerServer( config, metaProps, raftManager, time, metrics, + brokerMetrics, threadNamePrefix, offlineDirs, - controllerQuorumVotersFuture + controllerQuorumVotersFuture, + fatalFaultHandler, + metadataLoadingFaultHandler, + metadataApplyingFaultHandler )) } else { None } private val controller: Option[ControllerServer] = if (config.processRoles.contains(ControllerRole)) { + val controllerMetrics = new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry(), time) + val metadataFaultHandler = new LoggingFaultHandler("controller metadata", + () => controllerMetrics.incrementMetadataErrorCount()) + val fatalFaultHandler = new ProcessExitingFaultHandler() Some(new ControllerServer( metaProps, config, raftManager, time, metrics, + controllerMetrics, threadNamePrefix, controllerQuorumVotersFuture, KafkaRaftServer.configSchema, raftManager.apiVersions, - bootstrapMetadata + bootstrapMetadata, + metadataFaultHandler, + fatalFaultHandler )) } else { None @@ -114,6 +132,8 @@ class KafkaRaftServer( override def startup(): Unit = { Mx4jLoader.maybeLoad() + // Note that we startup `RaftManager` first so that the controller and broker + // can register listeners during initialization. raftManager.startup() controller.foreach(_.startup()) broker.foreach(_.startup()) @@ -123,6 +143,10 @@ class KafkaRaftServer( override def shutdown(): Unit = { broker.foreach(_.shutdown()) + // The order of shutdown for `RaftManager` and `ControllerServer` is backwards + // compared to `startup()`. This is because the `SocketServer` implementation that + // we rely on to receive requests is owned by `ControllerServer`, so we need it + // to stick around until graceful shutdown of `RaftManager` can be completed. raftManager.shutdown() controller.foreach(_.shutdown()) CoreUtils.swallow(AppInfoParser.unregisterAppInfo(Server.MetricsPrefix, config.brokerId.toString, metrics), this) @@ -137,8 +161,8 @@ class KafkaRaftServer( } object KafkaRaftServer { - val MetadataTopic = Topic.METADATA_TOPIC_NAME - val MetadataPartition = Topic.METADATA_TOPIC_PARTITION + val MetadataTopic = Topic.CLUSTER_METADATA_TOPIC_NAME + val MetadataPartition = Topic.CLUSTER_METADATA_TOPIC_PARTITION val MetadataTopicId = Uuid.METADATA_TOPIC_ID sealed trait ProcessRole @@ -182,16 +206,9 @@ object KafkaRaftServer { "If you intend to create a new broker, you should remove all data in your data directories (log.dirs).") } - // Load the bootstrap metadata file. In the case of an upgrade from older KRaft where there is no bootstrap metadata, - // read the IBP from config in order to bootstrap the equivalent metadata version. - def getUserDefinedIBPVersionOrThrow(): MetadataVersion = { - if (config.originals.containsKey(KafkaConfig.InterBrokerProtocolVersionProp)) { - MetadataVersion.fromVersionString(config.interBrokerProtocolVersionString) - } else { - throw new KafkaException(s"Cannot upgrade from KRaft version prior to 3.3 without first setting ${KafkaConfig.InterBrokerProtocolVersionProp} on each broker.") - } - } - val bootstrapMetadata = BootstrapMetadata.load(Paths.get(config.metadataLogDir), asJavaSupplier(() => getUserDefinedIBPVersionOrThrow())) + val bootstrapDirectory = new BootstrapDirectory(config.metadataLogDir, + Optional.ofNullable(config.interBrokerProtocolVersionString)) + val bootstrapMetadata = bootstrapDirectory.read() (metaProperties, bootstrapMetadata, offlineDirs.toSeq) } diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 2e728ce8173a8..ea94aaa1378b9 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -23,6 +23,8 @@ import org.apache.kafka.common.requests._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.server.common.MetadataVersion +import scala.collection.mutable + class ReplicaFetcherThread(name: String, leader: LeaderEndPoint, brokerConfig: KafkaConfig, @@ -41,6 +43,9 @@ class ReplicaFetcherThread(name: String, this.logIdent = logPrefix + // Visible for testing + private[server] val partitionsWithNewHighWatermark = mutable.Buffer[TopicPartition]() + override protected val isOffsetForLeaderEpochSupported: Boolean = metadataVersionSupplier().isOffsetForLeaderEpochSupported override protected def latestEpoch(topicPartition: TopicPartition): Option[Int] = { @@ -88,6 +93,11 @@ class ReplicaFetcherThread(name: String, } } + override def doWork(): Unit = { + super.doWork() + completeDelayedFetchRequests() + } + // process fetched data override def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, @@ -117,10 +127,16 @@ class ReplicaFetcherThread(name: String, // For the follower replica, we do not need to keep its segment base offset and physical position. // These values will be computed upon becoming leader or handling a preferred read replica fetch. - val followerHighWatermark = log.updateHighWatermark(partitionData.highWatermark) + var maybeUpdateHighWatermarkMessage = s"but did not update replica high watermark" + log.maybeUpdateHighWatermark(partitionData.highWatermark).foreach { newHighWatermark => + maybeUpdateHighWatermarkMessage = s"and updated replica high watermark to $newHighWatermark" + partitionsWithNewHighWatermark += topicPartition + } + log.maybeIncrementLogStartOffset(leaderLogStartOffset, LeaderOffsetIncremented) if (logTrace) - trace(s"Follower set replica high watermark for partition $topicPartition to $followerHighWatermark") + trace(s"Follower received high watermark ${partitionData.highWatermark} from the leader " + + s"$maybeUpdateHighWatermarkMessage for partition $topicPartition") // Traffic from both in-sync and out of sync replicas are accounted for in replication quota to ensure total replication // traffic doesn't exceed quota. @@ -135,6 +151,13 @@ class ReplicaFetcherThread(name: String, logAppendInfo } + private def completeDelayedFetchRequests(): Unit = { + if (partitionsWithNewHighWatermark.nonEmpty) { + replicaMgr.completeDelayedFetchRequests(partitionsWithNewHighWatermark.toSeq) + partitionsWithNewHighWatermark.clear() + } + } + def maybeWarnIfOversizedRecords(records: MemoryRecords, topicPartition: TopicPartition): Unit = { // oversized messages don't cause replication to fail from fetch request version 3 (KIP-74) if (metadataVersionSupplier().fetchRequestVersion <= 2 && records.sizeInBytes > 0 && records.validBytes <= 0) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index f60bd53a085e3..744e3780d8d2e 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -331,6 +331,15 @@ class ReplicaManager(val config: KafkaConfig, delayedFetchPurgatory.checkAndComplete(topicPartitionOperationKey) } + /** + * Complete any local follower fetches that have been unblocked since new data is available + * from the leader for one or more partitions. Should only be called by ReplicaFetcherThread + * after successfully replicating from the leader. + */ + private[server] def completeDelayedFetchRequests(topicPartitions: Seq[TopicPartition]): Unit = { + topicPartitions.foreach(tp => delayedFetchPurgatory.checkAndComplete(TopicPartitionOperationKey(tp))) + } + def stopReplicas(correlationId: Int, controllerId: Int, controllerEpoch: Int, @@ -1235,8 +1244,14 @@ class ReplicaManager(val config: KafkaConfig, partition.remoteReplicas.foreach { replica => val replicaState = replica.stateSnapshot - // Exclude replicas that don't have the requested offset (whether or not if they're in the ISR) - if (replicaState.logEndOffset >= fetchOffset && replicaState.logStartOffset <= fetchOffset) { + // Exclude replicas that are not in the ISR as the follower may lag behind. Worst case, the follower + // will continue to lag and the consumer will fall behind the produce. The leader will + // continuously pick the lagging follower when the consumer refreshes its preferred read replica. + // This can go on indefinitely. + if (partition.inSyncReplicaIds.contains(replica.brokerId) && + replicaState.logEndOffset >= fetchOffset && + replicaState.logStartOffset <= fetchOffset) { + replicaInfoSet.add(new DefaultReplicaView( replicaEndpoints.getOrElse(replica.brokerId, Node.noNode()), replicaState.logEndOffset, @@ -2142,7 +2157,6 @@ class ReplicaManager(val config: KafkaConfig, ): Unit = { stateChangeLogger.info(s"Transitioning ${localFollowers.size} partition(s) to " + "local followers.") - val shuttingDown = isShuttingDown.get() val partitionsToStartFetching = new mutable.HashMap[TopicPartition, Partition] val partitionsToStopFetching = new mutable.HashMap[TopicPartition, Boolean] val followerTopicSet = new mutable.HashSet[String] @@ -2151,28 +2165,24 @@ class ReplicaManager(val config: KafkaConfig, try { followerTopicSet.add(tp.topic) - if (shuttingDown) { - stateChangeLogger.trace(s"Unable to start fetching $tp with topic " + - s"ID ${info.topicId} because the replica manager is shutting down.") - } else { - // We always update the follower state. - // - This ensure that a replica with no leader can step down; - // - This also ensures that the local replica is created even if the leader - // is unavailable. This is required to ensure that we include the partition's - // high watermark in the checkpoint file (see KAFKA-1647). - val state = info.partition.toLeaderAndIsrPartitionState(tp, isNew) - val isNewLeaderEpoch = partition.makeFollower(state, offsetCheckpoints, Some(info.topicId)) - - if (isInControlledShutdown && (info.partition.leader == NO_LEADER || - !info.partition.isr.contains(config.brokerId))) { - // During controlled shutdown, replica with no leaders and replica - // where this broker is not in the ISR are stopped. - partitionsToStopFetching.put(tp, false) - } else if (isNewLeaderEpoch) { - // Otherwise, fetcher is restarted if the leader epoch has changed. - partitionsToStartFetching.put(tp, partition) - } + // We always update the follower state. + // - This ensure that a replica with no leader can step down; + // - This also ensures that the local replica is created even if the leader + // is unavailable. This is required to ensure that we include the partition's + // high watermark in the checkpoint file (see KAFKA-1647). + val state = info.partition.toLeaderAndIsrPartitionState(tp, isNew) + val isNewLeaderEpoch = partition.makeFollower(state, offsetCheckpoints, Some(info.topicId)) + + if (isInControlledShutdown && (info.partition.leader == NO_LEADER || + !info.partition.isr.contains(config.brokerId))) { + // During controlled shutdown, replica with no leaders and replica + // where this broker is not in the ISR are stopped. + partitionsToStopFetching.put(tp, false) + } else if (isNewLeaderEpoch) { + // Otherwise, fetcher is restarted if the leader epoch has changed. + partitionsToStartFetching.put(tp, partition) } + changedPartitions.add(partition) } catch { case e: KafkaStorageException => diff --git a/core/src/main/scala/kafka/server/RequestHandlerHelper.scala b/core/src/main/scala/kafka/server/RequestHandlerHelper.scala index a1aab617b3a28..5db595986efb3 100644 --- a/core/src/main/scala/kafka/server/RequestHandlerHelper.scala +++ b/core/src/main/scala/kafka/server/RequestHandlerHelper.scala @@ -95,10 +95,13 @@ class RequestHandlerHelper( def sendForwardedResponse(request: RequestChannel.Request, response: AbstractResponse): Unit = { - // For forwarded requests, we take the throttle time from the broker that - // the request was forwarded to - val throttleTimeMs = response.throttleTimeMs() - throttle(quotas.request, request, throttleTimeMs) + // For requests forwarded to the controller, we take the maximum of the local + // request throttle and the throttle sent by the controller in the response. + val controllerThrottleTimeMs = response.throttleTimeMs() + val requestThrottleTimeMs = maybeRecordAndGetThrottleTimeMs(request) + val appliedThrottleTimeMs = math.max(controllerThrottleTimeMs, requestThrottleTimeMs) + throttle(quotas.request, request, appliedThrottleTimeMs) + response.maybeSetThrottleTimeMs(appliedThrottleTimeMs) requestChannel.sendResponse(request, response, None) } diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala index fa0bc52d7aa01..cf7bf5aed9012 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala @@ -19,15 +19,17 @@ package kafka.server.metadata import java.util import java.util.concurrent.{CompletableFuture, TimeUnit} import java.util.function.Consumer - import kafka.metrics.KafkaMetricsGroup import org.apache.kafka.image.{MetadataDelta, MetadataImage} import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.queue.{EventQueue, KafkaEventQueue} import org.apache.kafka.raft.{Batch, BatchReader, LeaderAndEpoch, RaftClient} import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.fault.FaultHandler import org.apache.kafka.snapshot.SnapshotReader +import java.util.concurrent.atomic.AtomicBoolean + object BrokerMetadataListener { val MetadataBatchProcessingTimeUs = "MetadataBatchProcessingTimeUs" @@ -40,8 +42,23 @@ class BrokerMetadataListener( threadNamePrefix: Option[String], val maxBytesBetweenSnapshots: Long, val snapshotter: Option[MetadataSnapshotter], - brokerMetrics: BrokerServerMetrics + brokerMetrics: BrokerServerMetrics, + _metadataLoadingFaultHandler: FaultHandler ) extends RaftClient.Listener[ApiMessageAndVersion] with KafkaMetricsGroup { + + private val metadataFaultOccurred = new AtomicBoolean(false) + private val metadataLoadingFaultHandler: FaultHandler = new FaultHandler() { + override def handleFault(failureMessage: String, cause: Throwable): RuntimeException = { + // If the broker has any kind of error handling metadata records or publishing a new image + // we will disable taking new snapshots in order to preserve the local metadata log. Once we + // encounter a metadata processing error, the broker will be in an undetermined state. + if (metadataFaultOccurred.compareAndSet(false, true)) { + error("Disabling metadata snapshots until this broker is restarted.") + } + _metadataLoadingFaultHandler.handleFault(failureMessage, cause) + } + } + private val logContext = new LogContext(s"[BrokerMetadataListener id=$brokerId] ") private val log = logContext.logger(classOf[BrokerMetadataListener]) logIdent = logContext.logPrefix() @@ -109,34 +126,49 @@ class BrokerMetadataListener( extends EventQueue.FailureLoggingEvent(log) { override def run(): Unit = { val results = try { - val loadResults = loadBatches(_delta, reader, None, None, None) + val loadResults = loadBatches(_delta, reader, None, None, None, None) if (isDebugEnabled) { debug(s"Loaded new commits: $loadResults") } loadResults + } catch { + case e: Throwable => + metadataLoadingFaultHandler.handleFault(s"Unable to load metadata commits " + + s"from the BatchReader starting at base offset ${reader.baseOffset()}", e) + return } finally { reader.close() } - _publisher.foreach(publish) - // If we detected a change in metadata.version, generate a local snapshot - val metadataVersionChanged = Option(_delta.featuresDelta()).exists { featuresDelta => - featuresDelta.metadataVersionChange().isPresent + _bytesSinceLastSnapshot = _bytesSinceLastSnapshot + results.numBytes + if (shouldSnapshot()) { + maybeStartSnapshot() } - snapshotter.foreach { snapshotter => - _bytesSinceLastSnapshot = _bytesSinceLastSnapshot + results.numBytes - if (shouldSnapshot() || metadataVersionChanged) { - if (snapshotter.maybeStartSnapshot(_highestTimestamp, _delta.apply())) { - _bytesSinceLastSnapshot = 0L - } - } - } + _publisher.foreach(publish) } } private def shouldSnapshot(): Boolean = { - _bytesSinceLastSnapshot >= maxBytesBetweenSnapshots + (_bytesSinceLastSnapshot >= maxBytesBetweenSnapshots) || metadataVersionChanged() + } + + private def metadataVersionChanged(): Boolean = { + // The _publisher is empty before starting publishing, and we won't compute feature delta + // until we starting publishing + _publisher.nonEmpty && Option(_delta.featuresDelta()).exists { featuresDelta => + featuresDelta.metadataVersionChange().isPresent + } + } + + private def maybeStartSnapshot(): Unit = { + snapshotter.foreach { snapshotter => + if (metadataFaultOccurred.get()) { + trace("Not starting metadata snapshot since we previously had an error") + } else if (snapshotter.maybeStartSnapshot(_highestTimestamp, _delta.apply())) { + _bytesSinceLastSnapshot = 0L + } + } } /** @@ -148,19 +180,26 @@ class BrokerMetadataListener( class HandleSnapshotEvent(reader: SnapshotReader[ApiMessageAndVersion]) extends EventQueue.FailureLoggingEvent(log) { override def run(): Unit = { + val snapshotName = s"${reader.snapshotId().offset}-${reader.snapshotId().epoch}" try { - info(s"Loading snapshot ${reader.snapshotId().offset}-${reader.snapshotId().epoch}.") + info(s"Loading snapshot ${snapshotName}") _delta = new MetadataDelta(_image) // Discard any previous deltas. - val loadResults = loadBatches( - _delta, + val loadResults = loadBatches(_delta, reader, Some(reader.lastContainedLogTimestamp), Some(reader.lastContainedLogOffset), - Some(reader.lastContainedLogEpoch) - ) - _delta.finishSnapshot() - info(s"Loaded snapshot ${reader.snapshotId().offset}-${reader.snapshotId().epoch}: " + - s"$loadResults") + Some(reader.lastContainedLogEpoch), + Some(snapshotName)) + try { + _delta.finishSnapshot() + } catch { + case e: Throwable => metadataLoadingFaultHandler.handleFault( + s"Error finishing snapshot ${snapshotName}", e) + } + info(s"Loaded snapshot ${snapshotName}: ${loadResults}") + } catch { + case t: Throwable => metadataLoadingFaultHandler.handleFault("Uncaught exception while " + + s"loading broker metadata from Metadata snapshot ${snapshotName}", t) } finally { reader.close() } @@ -193,7 +232,8 @@ class BrokerMetadataListener( iterator: util.Iterator[Batch[ApiMessageAndVersion]], lastAppendTimestamp: Option[Long], lastCommittedOffset: Option[Long], - lastCommittedEpoch: Option[Int] + lastCommittedEpoch: Option[Int], + snapshotName: Option[String] ): BatchLoadResults = { val startTimeNs = time.nanoseconds() var numBatches = 0 @@ -212,12 +252,20 @@ class BrokerMetadataListener( trace(s"Metadata batch ${batch.lastOffset}: processing [${index + 1}/${batch.records.size}]:" + s" ${messageAndVersion.message}") } - - _highestOffset = lastCommittedOffset.getOrElse(batch.baseOffset() + index) - - delta.replay(highestMetadataOffset, epoch, messageAndVersion.message()) - numRecords += 1 - index += 1 + _highestOffset = lastCommittedOffset.getOrElse(batch.baseOffset() + index) + try { + delta.replay(highestMetadataOffset, epoch, messageAndVersion.message()) + } catch { + case e: Throwable => snapshotName match { + case None => metadataLoadingFaultHandler.handleFault( + s"Error replaying metadata log record at offset ${_highestOffset}", e) + case Some(name) => metadataLoadingFaultHandler.handleFault( + s"Error replaying record ${index} from snapshot ${name} at offset ${_highestOffset}", e) + } + } finally { + numRecords += 1 + index += 1 + } } numBytes = numBytes + batch.sizeInBytes() metadataBatchSizeHist.update(batch.records().size()) @@ -244,6 +292,9 @@ class BrokerMetadataListener( _publisher = Some(publisher) log.info(s"Starting to publish metadata events at offset $highestMetadataOffset.") try { + if (metadataVersionChanged()) { + maybeStartSnapshot() + } publish(publisher) future.complete(null) } catch { @@ -274,11 +325,21 @@ class BrokerMetadataListener( private def publish(publisher: MetadataPublisher): Unit = { val delta = _delta - _image = _delta.apply() + try { + _image = _delta.apply() + } catch { + case t: Throwable => + // If we cannot apply the delta, this publish event will throw and we will not publish a new image. + // The broker will continue applying metadata records and attempt to publish again. + throw metadataLoadingFaultHandler.handleFault(s"Error applying metadata delta $delta", t) + } + _delta = new MetadataDelta(_image) if (isDebugEnabled) { debug(s"Publishing new metadata delta $delta at offset ${_image.highestOffsetAndEpoch().offset}.") } + + // This publish call is done with its own try-catch and fault handler publisher.publish(delta, _image) // Update the metrics since the publisher handled the lastest image diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 212f188504e92..0192bb4afcfd7 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -31,6 +31,7 @@ import org.apache.kafka.common.internals.Topic import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta, TopicsImage} import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer import org.apache.kafka.server.authorizer.Authorizer +import org.apache.kafka.server.fault.FaultHandler import scala.collection.mutable @@ -94,15 +95,19 @@ object BrokerMetadataPublisher extends Logging { } } -class BrokerMetadataPublisher(conf: KafkaConfig, - metadataCache: KRaftMetadataCache, - logManager: LogManager, - replicaManager: ReplicaManager, - groupCoordinator: GroupCoordinator, - txnCoordinator: TransactionCoordinator, - clientQuotaMetadataManager: ClientQuotaMetadataManager, - dynamicConfigHandlers: Map[String, ConfigHandler], - private val _authorizer: Option[Authorizer]) extends MetadataPublisher with Logging { +class BrokerMetadataPublisher( + conf: KafkaConfig, + metadataCache: KRaftMetadataCache, + logManager: LogManager, + replicaManager: ReplicaManager, + groupCoordinator: GroupCoordinator, + txnCoordinator: TransactionCoordinator, + clientQuotaMetadataManager: ClientQuotaMetadataManager, + dynamicConfigHandlers: Map[String, ConfigHandler], + private val _authorizer: Option[Authorizer], + fatalFaultHandler: FaultHandler, + metadataPublishingFaultHandler: FaultHandler +) extends MetadataPublisher with Logging { logIdent = s"[BrokerMetadataPublisher id=${conf.nodeId}] " import BrokerMetadataPublisher._ @@ -125,8 +130,15 @@ class BrokerMetadataPublisher(conf: KafkaConfig, override def publish(delta: MetadataDelta, newImage: MetadataImage): Unit = { val highestOffsetAndEpoch = newImage.highestOffsetAndEpoch() + val deltaName = if (_firstPublish) { + s"initial MetadataDelta up to ${highestOffsetAndEpoch.offset}" + } else { + s"MetadataDelta up to ${highestOffsetAndEpoch.offset}" + } try { - trace(s"Publishing delta $delta with highest offset $highestOffsetAndEpoch") + if (isTraceEnabled) { + trace(s"Publishing delta $delta with highest offset $highestOffsetAndEpoch") + } // Publish the new metadata image to the metadata cache. metadataCache.setImage(newImage) @@ -151,37 +163,50 @@ class BrokerMetadataPublisher(conf: KafkaConfig, // Apply topic deltas. Option(delta.topicsDelta()).foreach { topicsDelta => - // Notify the replica manager about changes to topics. - replicaManager.applyDelta(topicsDelta, newImage) - - // Update the group coordinator of local changes - updateCoordinator( - newImage, - delta, - Topic.GROUP_METADATA_TOPIC_NAME, - groupCoordinator.onElection, - groupCoordinator.onResignation - ) - - // Update the transaction coordinator of local changes - updateCoordinator( - newImage, - delta, - Topic.TRANSACTION_STATE_TOPIC_NAME, - txnCoordinator.onElection, - txnCoordinator.onResignation - ) - - // Notify the group coordinator about deleted topics. - val deletedTopicPartitions = new mutable.ArrayBuffer[TopicPartition]() - topicsDelta.deletedTopicIds().forEach { id => - val topicImage = topicsDelta.image().getTopic(id) - topicImage.partitions().keySet().forEach { - id => deletedTopicPartitions += new TopicPartition(topicImage.name(), id) - } + try { + // Notify the replica manager about changes to topics. + replicaManager.applyDelta(topicsDelta, newImage) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error applying topics " + + s"delta in ${deltaName}", t) + } + try { + // Update the group coordinator of local changes + updateCoordinator(newImage, + delta, + Topic.GROUP_METADATA_TOPIC_NAME, + groupCoordinator.onElection, + groupCoordinator.onResignation) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating group " + + s"coordinator with local changes in ${deltaName}", t) + } + try { + // Update the transaction coordinator of local changes + updateCoordinator(newImage, + delta, + Topic.TRANSACTION_STATE_TOPIC_NAME, + txnCoordinator.onElection, + txnCoordinator.onResignation) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating txn " + + s"coordinator with local changes in ${deltaName}", t) } - if (deletedTopicPartitions.nonEmpty) { - groupCoordinator.handleDeletedPartitions(deletedTopicPartitions, RequestLocal.NoCaching) + try { + // Notify the group coordinator about deleted topics. + val deletedTopicPartitions = new mutable.ArrayBuffer[TopicPartition]() + topicsDelta.deletedTopicIds().forEach { id => + val topicImage = topicsDelta.image().getTopic(id) + topicImage.partitions().keySet().forEach { + id => deletedTopicPartitions += new TopicPartition(topicImage.name(), id) + } + } + if (deletedTopicPartitions.nonEmpty) { + groupCoordinator.handleDeletedPartitions(deletedTopicPartitions, RequestLocal.NoCaching) + } + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating group " + + s"coordinator with deleted partitions in ${deltaName}", t) } } @@ -191,39 +216,62 @@ class BrokerMetadataPublisher(conf: KafkaConfig, val props = newImage.configs().configProperties(resource) resource.`type`() match { case TOPIC => - // Apply changes to a topic's dynamic configuration. - info(s"Updating topic ${resource.name()} with new configuration : " + - toLoggableProps(resource, props).mkString(",")) - dynamicConfigHandlers(ConfigType.Topic). - processConfigChanges(resource.name(), props) + try { + // Apply changes to a topic's dynamic configuration. + info(s"Updating topic ${resource.name()} with new configuration : " + + toLoggableProps(resource, props).mkString(",")) + dynamicConfigHandlers(ConfigType.Topic). + processConfigChanges(resource.name(), props) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating topic " + + s"${resource.name()} with new configuration: ${toLoggableProps(resource, props).mkString(",")} " + + s"in ${deltaName}", t) + } case BROKER => if (resource.name().isEmpty) { - // Apply changes to "cluster configs" (also known as default BROKER configs). - // These are stored in KRaft with an empty name field. - info("Updating cluster configuration : " + - toLoggableProps(resource, props).mkString(",")) - dynamicConfigHandlers(ConfigType.Broker). - processConfigChanges(ConfigEntityName.Default, props) + try { + // Apply changes to "cluster configs" (also known as default BROKER configs). + // These are stored in KRaft with an empty name field. + info("Updating cluster configuration : " + + toLoggableProps(resource, props).mkString(",")) + dynamicConfigHandlers(ConfigType.Broker). + processConfigChanges(ConfigEntityName.Default, props) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating " + + s"cluster with new configuration: ${toLoggableProps(resource, props).mkString(",")} " + + s"in ${deltaName}", t) + } } else if (resource.name() == brokerId.toString) { - // Apply changes to this broker's dynamic configuration. - info(s"Updating broker $brokerId with new configuration : " + - toLoggableProps(resource, props).mkString(",")) - dynamicConfigHandlers(ConfigType.Broker). - processConfigChanges(resource.name(), props) - // When applying a per broker config (not a cluster config), we also - // reload any associated file. For example, if the ssl.keystore is still - // set to /tmp/foo, we still want to reload /tmp/foo in case its contents - // have changed. This doesn't apply to topic configs or cluster configs. - reloadUpdatedFilesWithoutConfigChange(props) + try { + // Apply changes to this broker's dynamic configuration. + info(s"Updating broker $brokerId with new configuration : " + + toLoggableProps(resource, props).mkString(",")) + dynamicConfigHandlers(ConfigType.Broker). + processConfigChanges(resource.name(), props) + // When applying a per broker config (not a cluster config), we also + // reload any associated file. For example, if the ssl.keystore is still + // set to /tmp/foo, we still want to reload /tmp/foo in case its contents + // have changed. This doesn't apply to topic configs or cluster configs. + reloadUpdatedFilesWithoutConfigChange(props) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating " + + s"broker with new configuration: ${toLoggableProps(resource, props).mkString(",")} " + + s"in ${deltaName}", t) + } } case _ => // nothing to do } } } - // Apply client quotas delta. - Option(delta.clientQuotasDelta()).foreach { clientQuotasDelta => - clientQuotaMetadataManager.update(clientQuotasDelta) + try { + // Apply client quotas delta. + Option(delta.clientQuotasDelta()).foreach { clientQuotasDelta => + clientQuotaMetadataManager.update(clientQuotasDelta) + } + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating client " + + s"quotas in ${deltaName}", t) } // Apply changes to ACLs. This needs to be handled carefully because while we are @@ -235,20 +283,30 @@ class BrokerMetadataPublisher(conf: KafkaConfig, Option(delta.aclsDelta()).foreach( aclsDelta => _authorizer match { case Some(authorizer: ClusterMetadataAuthorizer) => if (aclsDelta.isSnapshotDelta) { - // If the delta resulted from a snapshot load, we want to apply the new changes - // all at once using ClusterMetadataAuthorizer#loadSnapshot. If this is the - // first snapshot load, it will also complete the futures returned by - // Authorizer#start (which we wait for before processing RPCs). - authorizer.loadSnapshot(newImage.acls().acls()) + try { + // If the delta resulted from a snapshot load, we want to apply the new changes + // all at once using ClusterMetadataAuthorizer#loadSnapshot. If this is the + // first snapshot load, it will also complete the futures returned by + // Authorizer#start (which we wait for before processing RPCs). + authorizer.loadSnapshot(newImage.acls().acls()) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error loading " + + s"authorizer snapshot in ${deltaName}", t) + } } else { - // Because the changes map is a LinkedHashMap, the deltas will be returned in - // the order they were performed. - aclsDelta.changes().entrySet().forEach(e => - if (e.getValue.isPresent) { - authorizer.addAcl(e.getKey, e.getValue.get()) - } else { - authorizer.removeAcl(e.getKey) - }) + try { + // Because the changes map is a LinkedHashMap, the deltas will be returned in + // the order they were performed. + aclsDelta.changes().entrySet().forEach(e => + if (e.getValue.isPresent) { + authorizer.addAcl(e.getKey, e.getValue.get()) + } else { + authorizer.removeAcl(e.getKey) + }) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error loading " + + s"authorizer changes in ${deltaName}", t) + } } case _ => // No ClusterMetadataAuthorizer is configured. There is nothing to do. }) @@ -258,8 +316,8 @@ class BrokerMetadataPublisher(conf: KafkaConfig, } publishedOffsetAtomic.set(newImage.highestOffsetAndEpoch().offset) } catch { - case t: Throwable => error(s"Error publishing broker metadata at $highestOffsetAndEpoch", t) - throw t + case t: Throwable => metadataPublishingFaultHandler.handleFault("Uncaught exception while " + + s"publishing broker metadata from ${deltaName}", t) } finally { _firstPublish = false } @@ -282,7 +340,7 @@ class BrokerMetadataPublisher(conf: KafkaConfig, * @param resignation function to call on resignation; the first parameter is the partition id; * the second parameter is the leader epoch */ - private def updateCoordinator( + def updateCoordinator( image: MetadataImage, delta: MetadataDelta, topicName: String, @@ -317,38 +375,60 @@ class BrokerMetadataPublisher(conf: KafkaConfig, } private def initializeManagers(): Unit = { - // Start log manager, which will perform (potentially lengthy) - // recovery-from-unclean-shutdown if required. - logManager.startup(metadataCache.getAllTopics()) - - // Make the LogCleaner available for reconfiguration. We can't do this prior to this - // point because LogManager#startup creates the LogCleaner object, if - // log.cleaner.enable is true. TODO: improve this (see KAFKA-13610) - Option(logManager.cleaner).foreach(conf.dynamicConfig.addBrokerReconfigurable) - - // Start the replica manager. - replicaManager.startup() - - // Start the group coordinator. - groupCoordinator.startup(() => metadataCache.numPartitions( - Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(conf.offsetsTopicPartitions)) - - // Start the transaction coordinator. - txnCoordinator.startup(() => metadataCache.numPartitions( - Topic.TRANSACTION_STATE_TOPIC_NAME).getOrElse(conf.transactionTopicPartitions)) + try { + // Start log manager, which will perform (potentially lengthy) + // recovery-from-unclean-shutdown if required. + logManager.startup(metadataCache.getAllTopics()) + + // Make the LogCleaner available for reconfiguration. We can't do this prior to this + // point because LogManager#startup creates the LogCleaner object, if + // log.cleaner.enable is true. TODO: improve this (see KAFKA-13610) + Option(logManager.cleaner).foreach(conf.dynamicConfig.addBrokerReconfigurable) + } catch { + case t: Throwable => fatalFaultHandler.handleFault("Error starting LogManager", t) + } + try { + // Start the replica manager. + replicaManager.startup() + } catch { + case t: Throwable => fatalFaultHandler.handleFault("Error starting ReplicaManager", t) + } + try { + // Start the group coordinator. + groupCoordinator.startup(() => metadataCache.numPartitions( + Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(conf.offsetsTopicPartitions)) + } catch { + case t: Throwable => fatalFaultHandler.handleFault("Error starting GroupCoordinator", t) + } + try { + // Start the transaction coordinator. + txnCoordinator.startup(() => metadataCache.numPartitions( + Topic.TRANSACTION_STATE_TOPIC_NAME).getOrElse(conf.transactionTopicPartitions)) + } catch { + case t: Throwable => fatalFaultHandler.handleFault("Error starting TransactionCoordinator", t) + } } private def finishInitializingReplicaManager(newImage: MetadataImage): Unit = { - // Delete log directories which we're not supposed to have, according to the - // latest metadata. This is only necessary to do when we're first starting up. If - // we have to load a snapshot later, these topics will appear in deletedTopicIds. - val strayPartitions = findStrayPartitions(brokerId, newImage.topics, logManager.allLogs) - if (strayPartitions.nonEmpty) { - replicaManager.deleteStrayReplicas(strayPartitions) + try { + // Delete log directories which we're not supposed to have, according to the + // latest metadata. This is only necessary to do when we're first starting up. If + // we have to load a snapshot later, these topics will appear in deletedTopicIds. + val strayPartitions = findStrayPartitions(brokerId, newImage.topics, logManager.allLogs) + if (strayPartitions.nonEmpty) { + replicaManager.deleteStrayReplicas(strayPartitions) + } + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error deleting stray " + + "partitions during startup", t) } - - // Make sure that the high water mark checkpoint thread is running for the replica - // manager. - replicaManager.startHighWatermarkCheckPointThread() - } + try { + // Make sure that the high water mark checkpoint thread is running for the replica + // manager. + replicaManager.startHighWatermarkCheckPointThread() + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error starting high " + + "watermark checkpoint thread during startup", t) + } +} } diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala index b5179c32f1416..2a236ca749706 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataSnapshotter.scala @@ -24,10 +24,32 @@ import org.apache.kafka.queue.{EventQueue, KafkaEventQueue} import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.snapshot.SnapshotWriter +import java.util.function.Consumer + trait SnapshotWriterBuilder { def build(committedOffset: Long, committedEpoch: Int, - lastContainedLogTime: Long): SnapshotWriter[ApiMessageAndVersion] + lastContainedLogTime: Long): Option[SnapshotWriter[ApiMessageAndVersion]] +} + +/** + * The RecordListConsumer takes as input a potentially long list of records, and feeds the + * SnapshotWriter a series of smaller lists of records. + * + * Note: from the perspective of Kafka, the snapshot file is really just a list of records, + * and we don't care about batches. Batching is irrelevant to the meaning of the snapshot. + */ +class RecordListConsumer( + val maxRecordsInBatch: Int, + val writer: SnapshotWriter[ApiMessageAndVersion] +) extends Consumer[java.util.List[ApiMessageAndVersion]] { + override def accept(messages: java.util.List[ApiMessageAndVersion]): Unit = { + var i = 0 + while (i < messages.size()) { + writer.append(messages.subList(i, Math.min(i + maxRecordsInBatch, messages.size()))); + i += maxRecordsInBatch + } + } } class BrokerMetadataSnapshotter( @@ -36,6 +58,16 @@ class BrokerMetadataSnapshotter( threadNamePrefix: Option[String], writerBuilder: SnapshotWriterBuilder ) extends Logging with MetadataSnapshotter { + /** + * The maximum number of records we will put in each batch. + * + * From the perspective of the Raft layer, the limit on batch size is specified in terms of + * bytes, not number of records. @See {@link KafkaRaftClient#MAX_BATCH_SIZE_BYTES} for details. + * However, it's more convenient to limit the batch size here in terms of number of records. + * So we chose a low number that will not cause problems. + */ + private val maxRecordsInBatch = 1024 + private val logContext = new LogContext(s"[BrokerMetadataSnapshotter id=$brokerId] ") logIdent = logContext.logPrefix() @@ -51,29 +83,37 @@ class BrokerMetadataSnapshotter( val eventQueue = new KafkaEventQueue(time, logContext, threadNamePrefix.getOrElse("")) override def maybeStartSnapshot(lastContainedLogTime: Long, image: MetadataImage): Boolean = synchronized { - if (_currentSnapshotOffset == -1L) { + if (_currentSnapshotOffset != -1) { + info(s"Declining to create a new snapshot at ${image.highestOffsetAndEpoch()} because " + + s"there is already a snapshot in progress at offset ${_currentSnapshotOffset}") + false + } else { val writer = writerBuilder.build( image.highestOffsetAndEpoch().offset, image.highestOffsetAndEpoch().epoch, lastContainedLogTime ) - _currentSnapshotOffset = image.highestOffsetAndEpoch().offset - info(s"Creating a new snapshot at offset ${_currentSnapshotOffset}...") - eventQueue.append(new CreateSnapshotEvent(image, writer)) - true - } else { - warn(s"Declining to create a new snapshot at ${image.highestOffsetAndEpoch()} because " + - s"there is already a snapshot in progress at offset ${_currentSnapshotOffset}") - false + if (writer.nonEmpty) { + _currentSnapshotOffset = image.highestOffsetAndEpoch().offset + info(s"Creating a new snapshot at offset ${_currentSnapshotOffset}...") + eventQueue.append(new CreateSnapshotEvent(image, writer.get)) + true + } else { + info(s"Declining to create a new snapshot at ${image.highestOffsetAndEpoch()} because " + + s"there is already a snapshot at offset ${image.highestOffsetAndEpoch().offset}") + false + } } } class CreateSnapshotEvent(image: MetadataImage, writer: SnapshotWriter[ApiMessageAndVersion]) extends EventQueue.Event { + override def run(): Unit = { try { - image.write(writer.append(_)) + val consumer = new RecordListConsumer(maxRecordsInBatch, writer) + image.write(consumer) writer.freeze() } finally { try { diff --git a/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala b/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala index 0db6f0071c486..3e68ae85f9232 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala @@ -28,6 +28,8 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable val lastAppliedRecordOffset: AtomicLong = new AtomicLong(0) val lastAppliedRecordTimestamp: AtomicLong = new AtomicLong(0) + val metadataLoadErrorCount: AtomicLong = new AtomicLong(0) + val metadataApplyErrorCount: AtomicLong = new AtomicLong(0) val lastAppliedRecordOffsetName = metrics.metricName( "last-applied-record-offset", @@ -47,6 +49,18 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable "The difference between now and the timestamp of the last record from the cluster metadata partition that was applied by the broker" ) + val metadataLoadErrorCountName = metrics.metricName( + "metadata-load-error-count", + metricGroupName, + "The number of errors encountered by the BrokerMetadataListener while loading the metadata log and generating a new MetadataDelta based on it." + ) + + val metadataApplyErrorCountName = metrics.metricName( + "metadata-apply-error-count", + metricGroupName, + "The number of errors encountered by the BrokerMetadataPublisher while applying a new MetadataImage based on the latest MetadataDelta." + ) + addMetric(metrics, lastAppliedRecordOffsetName) { _ => lastAppliedRecordOffset.get } @@ -59,11 +73,21 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable now - lastAppliedRecordTimestamp.get } + addMetric(metrics, metadataLoadErrorCountName) { _ => + metadataLoadErrorCount.get + } + + addMetric(metrics, metadataApplyErrorCountName) { _ => + metadataApplyErrorCount.get + } + override def close(): Unit = { List( lastAppliedRecordOffsetName, lastAppliedRecordTimestampName, - lastAppliedRecordLagMsName + lastAppliedRecordLagMsName, + metadataLoadErrorCountName, + metadataApplyErrorCountName ).foreach(metrics.removeMetric) } } diff --git a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala index ae2e65235739b..525772115037e 100644 --- a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala +++ b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala @@ -229,7 +229,7 @@ class KRaftMetadataCache(val brokerId: Int) extends MetadataCache with Logging w override def getPartitionInfo(topicName: String, partitionId: Int): Option[UpdateMetadataPartitionState] = { Option(_currentImage.topics().getTopic(topicName)). - flatMap(topic => Some(topic.partitions().get(partitionId))). + flatMap(topic => Option(topic.partitions().get(partitionId))). flatMap(partition => Some(new UpdateMetadataPartitionState(). setTopicName(topicName). setPartitionIndex(partitionId). diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index b57342ff29baf..c82523eff4b6b 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -31,6 +31,7 @@ import org.apache.kafka.common.protocol.ByteBufferAccessor import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.MetadataRecordSerde +import org.apache.kafka.metadata.bootstrap.BootstrapDirectory import org.apache.kafka.snapshot.Snapshots import scala.jdk.CollectionConverters._ @@ -253,8 +254,12 @@ object DumpLogSegments { val startOffset = file.getName.split("\\.")(0).toLong println(s"Log starting offset: $startOffset") } else if (file.getName.endsWith(Snapshots.SUFFIX)) { - val path = Snapshots.parse(file.toPath).get() - println(s"Snapshot end offset: ${path.snapshotId.offset}, epoch: ${path.snapshotId.epoch}") + if (file.getName == BootstrapDirectory.BINARY_BOOTSTRAP_FILENAME) { + println("KRaft bootstrap snapshot") + } else { + val path = Snapshots.parse(file.toPath).get() + println(s"Snapshot end offset: ${path.snapshotId.offset}, epoch: ${path.snapshotId.epoch}") + } } val fileRecords = FileRecords.open(file, false).slice(0, maxBytes) try { diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala index a96275cc27cd1..0f798e24fc215 100644 --- a/core/src/main/scala/kafka/tools/StorageTool.scala +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -26,9 +26,10 @@ import net.sourceforge.argparse4j.impl.Arguments.{store, storeTrue} import net.sourceforge.argparse4j.inf.Namespace import org.apache.kafka.common.Uuid import org.apache.kafka.common.utils.Utils -import org.apache.kafka.controller.BootstrapMetadata +import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata} import org.apache.kafka.server.common.MetadataVersion +import java.util.Optional import scala.collection.mutable object StorageTool extends Logging { @@ -47,7 +48,7 @@ object StorageTool extends Logging { case "format" => val directories = configToLogDirectories(config.get) val clusterId = namespace.getString("cluster_id") - val metadataVersion = getMetadataVersion(namespace) + val metadataVersion = getMetadataVersion(namespace, Option(config.get.interBrokerProtocolVersionString)) if (!metadataVersion.isKRaftSupported) { throw new TerseFailure(s"Must specify a valid KRaft metadata version of at least 3.0.") } @@ -113,10 +114,18 @@ object StorageTool extends Logging { def configToSelfManagedMode(config: KafkaConfig): Boolean = config.processRoles.nonEmpty - def getMetadataVersion(namespace: Namespace): MetadataVersion = { + def getMetadataVersion( + namespace: Namespace, + defaultVersionString: Option[String] + ): MetadataVersion = { + val defaultValue = defaultVersionString match { + case Some(versionString) => MetadataVersion.fromVersionString(versionString) + case None => MetadataVersion.latest() + } + Option(namespace.getString("release_version")) .map(ver => MetadataVersion.fromVersionString(ver)) - .getOrElse(MetadataVersion.latest()) + .getOrElse(defaultValue) } def infoCommand(stream: PrintStream, selfManagedMode: Boolean, directories: Seq[String]): Int = { @@ -233,7 +242,7 @@ object StorageTool extends Logging { if (!Files.isDirectory(Paths.get(directory)) || !Files.exists(Paths.get(directory, "meta.properties"))) { true } else if (!ignoreFormatted) { - throw new TerseFailure(s"Log directory ${directory} is already formatted. " + + throw new TerseFailure(s"Log directory $directory is already formatted. " + "Use --ignore-formatted to ignore this directory and format the others.") } else { false @@ -247,14 +256,15 @@ object StorageTool extends Logging { Files.createDirectories(Paths.get(directory)) } catch { case e: Throwable => throw new TerseFailure(s"Unable to create storage " + - s"directory ${directory}: ${e.getMessage}") + s"directory $directory: ${e.getMessage}") } val metaPropertiesPath = Paths.get(directory, "meta.properties") val checkpoint = new BrokerMetadataCheckpoint(metaPropertiesPath.toFile) checkpoint.write(metaProperties.toProperties) - val bootstrapMetadata = BootstrapMetadata.create(metadataVersion) - BootstrapMetadata.write(bootstrapMetadata, Paths.get(directory)) + val bootstrapMetadata = BootstrapMetadata.fromVersion(metadataVersion, "format command") + val bootstrapDirectory = new BootstrapDirectory(directory, Optional.empty()) + bootstrapDirectory.writeBinaryFile(bootstrapMetadata) stream.println(s"Formatting ${directory} with metadata.version ${metadataVersion}.") }) diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index a72784c469ad6..ef97b6ccdaa2f 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -299,11 +299,7 @@ object TestRaftServer extends Logging { out.writeByteArray(data) } - override def read(input: protocol.Readable, size: Int): Array[Byte] = { - val data = new Array[Byte](size) - input.readArray(data) - data - } + override def read(input: protocol.Readable, size: Int): Array[Byte] = input.readArray(size) } private class LatencyHistogram( diff --git a/core/src/main/scala/kafka/utils/PasswordEncoder.scala b/core/src/main/scala/kafka/utils/PasswordEncoder.scala index f748a455c62bf..3373223e36f1c 100644 --- a/core/src/main/scala/kafka/utils/PasswordEncoder.scala +++ b/core/src/main/scala/kafka/utils/PasswordEncoder.scala @@ -38,6 +38,33 @@ object PasswordEncoder { val IterationsProp = "iterations" val EncyrptedPasswordProp = "encryptedPassword" val PasswordLengthProp = "passwordLength" + + def encrypting(secret: Password, + keyFactoryAlgorithm: Option[String], + cipherAlgorithm: String, + keyLength: Int, + iterations: Int): EncryptingPasswordEncoder = { + new EncryptingPasswordEncoder(secret, keyFactoryAlgorithm, cipherAlgorithm, keyLength, iterations) + } + + def noop(): NoOpPasswordEncoder = { + new NoOpPasswordEncoder() + } +} + +trait PasswordEncoder { + def encode(password: Password): String + def decode(encodedPassword: String): Password + + private[utils] def base64Decode(encoded: String): Array[Byte] = Base64.getDecoder.decode(encoded) +} + +/** + * A password encoder that does not modify the given password. This is used in KRaft mode only. + */ +class NoOpPasswordEncoder extends PasswordEncoder { + override def encode(password: Password): String = password.value() + override def decode(encodedPassword: String): Password = new Password(encodedPassword) } /** @@ -55,16 +82,18 @@ object PasswordEncoder { * The values used for encoding are stored along with the encoded password and the stored values are used for decoding. * */ -class PasswordEncoder(secret: Password, - keyFactoryAlgorithm: Option[String], - cipherAlgorithm: String, - keyLength: Int, - iterations: Int) extends Logging { +class EncryptingPasswordEncoder( + secret: Password, + keyFactoryAlgorithm: Option[String], + cipherAlgorithm: String, + keyLength: Int, + iterations: Int +) extends PasswordEncoder with Logging { private val secureRandom = new SecureRandom private val cipherParamsEncoder = cipherParamsInstance(cipherAlgorithm) - def encode(password: Password): String = { + override def encode(password: Password): String = { val salt = new Array[Byte](256) secureRandom.nextBytes(salt) val cipher = Cipher.getInstance(cipherAlgorithm) @@ -84,7 +113,7 @@ class PasswordEncoder(secret: Password, encryptedMap.map { case (k, v) => s"$k:$v" }.mkString(",") } - def decode(encodedPassword: String): Password = { + override def decode(encodedPassword: String): Password = { val params = CoreUtils.parseCsvMap(encodedPassword) val keyFactoryAlg = params(KeyFactoryAlgorithmProp) val cipherAlg = params(CipherAlgorithmProp) @@ -131,8 +160,6 @@ class PasswordEncoder(secret: Password, private def base64Encode(bytes: Array[Byte]): String = Base64.getEncoder.encodeToString(bytes) - private[utils] def base64Decode(encoded: String): Array[Byte] = Base64.getDecoder.decode(encoded) - private def cipherParamsInstance(cipherAlgorithm: String): CipherParamsEncoder = { val aesPattern = "AES/(.*)/.*".r cipherAlgorithm match { diff --git a/core/src/test/java/kafka/test/ClusterInstance.java b/core/src/test/java/kafka/test/ClusterInstance.java index a7052857c364f..9058508fa94f7 100644 --- a/core/src/test/java/kafka/test/ClusterInstance.java +++ b/core/src/test/java/kafka/test/ClusterInstance.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.Set; public interface ClusterInstance { @@ -50,6 +51,18 @@ default boolean isKRaftTest() { */ ClusterConfig config(); + /** + * Return the set of all controller IDs configured for this test. For kraft, this + * will return only the nodes which have the "controller" role enabled in `process.roles`. + * For zookeeper, this will return all broker IDs since they are all eligible controllers. + */ + Set controllerIds(); + + /** + * Return the set of all broker IDs configured for this test. + */ + Set brokerIds(); + /** * The listener for this cluster as configured by {@link ClusterTest} or by {@link ClusterConfig}. If * unspecified by those sources, this will return the listener for the default security protocol PLAINTEXT diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java index cef71042d3f92..f5c281ff24967 100644 --- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java @@ -42,6 +42,7 @@ import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; @@ -182,6 +183,10 @@ public Map brokerFeatures() { )); } + public Collection controllerServers() { + return controllers().collect(Collectors.toList()); + } + @Override public ClusterType clusterType() { return ClusterType.RAFT; @@ -192,6 +197,20 @@ public ClusterConfig config() { return clusterConfig; } + @Override + public Set controllerIds() { + return controllers() + .map(controllerServer -> controllerServer.config().nodeId()) + .collect(Collectors.toSet()); + } + + @Override + public Set brokerIds() { + return brokers() + .map(brokerServer -> brokerServer.config().nodeId()) + .collect(Collectors.toSet()); + } + @Override public KafkaClusterTestKit getUnderlying() { return clusterReference.get(); @@ -252,11 +271,11 @@ private BrokerServer findBrokerOrThrow(int brokerId) { .orElseThrow(() -> new IllegalArgumentException("Unknown brokerId " + brokerId)); } - private Stream brokers() { + public Stream brokers() { return clusterReference.get().brokers().values().stream(); } - private Stream controllers() { + public Stream controllers() { return clusterReference.get().controllers().values().stream(); } diff --git a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java index d8375b012796b..18a85e2d7bf66 100644 --- a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java @@ -45,6 +45,7 @@ import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -257,6 +258,18 @@ public ClusterConfig config() { return config; } + @Override + public Set controllerIds() { + return brokerIds(); + } + + @Override + public Set brokerIds() { + return servers() + .map(brokerServer -> brokerServer.config().nodeId()) + .collect(Collectors.toSet()); + } + @Override public IntegrationTestHarness getUnderlying() { return clusterReference.get(); diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index c961d71bbe588..417c083457f49 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -24,6 +24,7 @@ import kafka.server.KafkaConfig$; import kafka.server.KafkaRaftServer; import kafka.server.MetaProperties; +import kafka.server.metadata.BrokerServerMetrics$; import kafka.tools.StorageTool; import kafka.utils.Logging; import org.apache.kafka.clients.CommonClientConfigs; @@ -34,12 +35,14 @@ import org.apache.kafka.common.utils.ThreadUtils; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.controller.BootstrapMetadata; import org.apache.kafka.controller.Controller; +import org.apache.kafka.controller.MockControllerMetrics; import org.apache.kafka.metadata.MetadataRecordSerde; +import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; import org.apache.kafka.raft.RaftConfig; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.fault.MockFaultHandler; import org.apache.kafka.test.TestUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -115,6 +118,8 @@ public void close() { public static class Builder { private TestKitNodes nodes; private Map configProps = new HashMap<>(); + private MockFaultHandler metadataFaultHandler = new MockFaultHandler("metadataFaultHandler"); + private MockFaultHandler fatalFaultHandler = new MockFaultHandler("fatalFaultHandler"); public Builder(TestKitNodes nodes) { this.nodes = nodes; @@ -125,6 +130,11 @@ public Builder setConfigProp(String key, String value) { return this; } + public Builder setMetadataFaultHandler(MockFaultHandler metadataFaultHandler) { + this.metadataFaultHandler = metadataFaultHandler; + return this; + } + public KafkaClusterTestKit build() throws Exception { Map controllers = new HashMap<>(); Map brokers = new HashMap<>(); @@ -176,7 +186,8 @@ public KafkaClusterTestKit build() throws Exception { String threadNamePrefix = String.format("controller%d_", node.id()); MetaProperties metaProperties = MetaProperties.apply(nodes.clusterId().toString(), node.id()); TopicPartition metadataPartition = new TopicPartition(KafkaRaftServer.MetadataTopic(), 0); - BootstrapMetadata bootstrapMetadata = BootstrapMetadata.create(nodes.bootstrapMetadataVersion()); + BootstrapMetadata bootstrapMetadata = BootstrapMetadata. + fromVersion(nodes.bootstrapMetadataVersion(), "testkit"); KafkaRaftManager raftManager = new KafkaRaftManager<>( metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftServer.MetadataTopicId(), Time.SYSTEM, new Metrics(), Option.apply(threadNamePrefix), connectFutureManager.future); @@ -186,11 +197,14 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS raftManager, Time.SYSTEM, new Metrics(), + new MockControllerMetrics(), Option.apply(threadNamePrefix), connectFutureManager.future, KafkaRaftServer.configSchema(), raftManager.apiVersions(), - bootstrapMetadata + bootstrapMetadata, + metadataFaultHandler, + fatalFaultHandler ); controllers.put(node.id(), controller); controller.socketServerFirstBoundPortFuture().whenComplete((port, e) -> { @@ -240,15 +254,20 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS Time.SYSTEM, new Metrics(), Option.apply(threadNamePrefix), connectFutureManager.future); raftManagers.put(node.id(), raftManager); } + Metrics metrics = new Metrics(); BrokerServer broker = new BrokerServer( config, nodes.brokerProperties(node.id()), raftManager, Time.SYSTEM, - new Metrics(), + metrics, + BrokerServerMetrics$.MODULE$.apply(metrics), Option.apply(threadNamePrefix), JavaConverters.asScalaBuffer(Collections.emptyList()).toSeq(), - connectFutureManager.future + connectFutureManager.future, + fatalFaultHandler, + metadataFaultHandler, + metadataFaultHandler ); brokers.put(node.id(), broker); } @@ -257,15 +276,15 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS executorService.shutdownNow(); executorService.awaitTermination(5, TimeUnit.MINUTES); } - for (ControllerServer controller : controllers.values()) { - controller.shutdown(); - } for (BrokerServer brokerServer : brokers.values()) { brokerServer.shutdown(); } for (KafkaRaftManager raftManager : raftManagers.values()) { raftManager.shutdown(); } + for (ControllerServer controller : controllers.values()) { + controller.shutdown(); + } connectFutureManager.close(); if (baseDirectory != null) { Utils.delete(baseDirectory); @@ -273,7 +292,8 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS throw e; } return new KafkaClusterTestKit(executorService, nodes, controllers, - brokers, raftManagers, connectFutureManager, baseDirectory); + brokers, raftManagers, connectFutureManager, baseDirectory, + metadataFaultHandler, fatalFaultHandler); } private String listeners(int node) { @@ -314,14 +334,20 @@ static private void setupNodeDirectories(File baseDirectory, private final Map> raftManagers; private final ControllerQuorumVotersFutureManager controllerQuorumVotersFutureManager; private final File baseDirectory; - - private KafkaClusterTestKit(ExecutorService executorService, - TestKitNodes nodes, - Map controllers, - Map brokers, - Map> raftManagers, - ControllerQuorumVotersFutureManager controllerQuorumVotersFutureManager, - File baseDirectory) { + private final MockFaultHandler metadataFaultHandler; + private final MockFaultHandler fatalFaultHandler; + + private KafkaClusterTestKit( + ExecutorService executorService, + TestKitNodes nodes, + Map controllers, + Map brokers, + Map> raftManagers, + ControllerQuorumVotersFutureManager controllerQuorumVotersFutureManager, + File baseDirectory, + MockFaultHandler metadataFaultHandler, + MockFaultHandler fatalFaultHandler + ) { this.executorService = executorService; this.nodes = nodes; this.controllers = controllers; @@ -329,6 +355,8 @@ private KafkaClusterTestKit(ExecutorService executorService, this.raftManagers = raftManagers; this.controllerQuorumVotersFutureManager = controllerQuorumVotersFutureManager; this.baseDirectory = baseDirectory; + this.metadataFaultHandler = metadataFaultHandler; + this.fatalFaultHandler = fatalFaultHandler; } public void format() throws Exception { @@ -365,7 +393,7 @@ private void formatNodeAndLog(MetaProperties properties, String metadataLogDir, StorageTool.formatCommand(out, JavaConverters.asScalaBuffer(Collections.singletonList(metadataLogDir)).toSeq(), properties, - MetadataVersion.MINIMUM_KRAFT_VERSION, + MetadataVersion.MINIMUM_BOOTSTRAP_VERSION, false); } finally { for (String line : stream.toString().split(String.format("%n"))) { @@ -381,12 +409,15 @@ private void formatNodeAndLog(MetaProperties properties, String metadataLogDir, public void startup() throws ExecutionException, InterruptedException { List> futures = new ArrayList<>(); try { - for (ControllerServer controller : controllers.values()) { - futures.add(executorService.submit(controller::startup)); - } + // Note the startup order here is chosen to be consistent with + // `KafkaRaftServer`. See comments in that class for an explanation. + for (KafkaRaftManager raftManager : raftManagers.values()) { futures.add(controllerQuorumVotersFutureManager.future.thenRunAsync(raftManager::startup)); } + for (ControllerServer controller : controllers.values()) { + futures.add(executorService.submit(controller::startup)); + } for (BrokerServer broker : brokers.values()) { futures.add(executorService.submit(broker::startup)); } @@ -486,6 +517,10 @@ public void close() throws Exception { List>> futureEntries = new ArrayList<>(); try { controllerQuorumVotersFutureManager.close(); + + // Note the shutdown order here is chosen to be consistent with + // `KafkaRaftServer`. See comments in that class for an explanation. + for (Entry entry : brokers.entrySet()) { int brokerId = entry.getKey(); BrokerServer broker = entry.getValue(); @@ -494,14 +529,6 @@ public void close() throws Exception { } waitForAllFutures(futureEntries); futureEntries.clear(); - for (Entry entry : controllers.entrySet()) { - int controllerId = entry.getKey(); - ControllerServer controller = entry.getValue(); - futureEntries.add(new SimpleImmutableEntry<>("controller" + controllerId, - executorService.submit(controller::shutdown))); - } - waitForAllFutures(futureEntries); - futureEntries.clear(); for (Entry> entry : raftManagers.entrySet()) { int raftManagerId = entry.getKey(); KafkaRaftManager raftManager = entry.getValue(); @@ -510,6 +537,14 @@ public void close() throws Exception { } waitForAllFutures(futureEntries); futureEntries.clear(); + for (Entry entry : controllers.entrySet()) { + int controllerId = entry.getKey(); + ControllerServer controller = entry.getValue(); + futureEntries.add(new SimpleImmutableEntry<>("controller" + controllerId, + executorService.submit(controller::shutdown))); + } + waitForAllFutures(futureEntries); + futureEntries.clear(); Utils.delete(baseDirectory); } catch (Exception e) { for (Entry> entry : futureEntries) { @@ -520,6 +555,8 @@ public void close() throws Exception { executorService.shutdownNow(); executorService.awaitTermination(5, TimeUnit.MINUTES); } + metadataFaultHandler.maybeRethrowFirstException(); + fatalFaultHandler.maybeRethrowFirstException(); } private void waitForAllFutures(List>> futureEntries) diff --git a/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala b/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala index 3082babd06fa0..ea4c748da92a8 100644 --- a/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala @@ -33,6 +33,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.MetadataResponse import org.apache.kafka.common.security.auth.SecurityProtocol import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -470,6 +471,30 @@ class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging wi TestUtils.verifyTopicDeletion(zkClientOrNull, testTopicName, 1, brokers) } + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testTopicWithCollidingCharDeletionAndCreateAgain(quorum: String): Unit = { + // create the topic with colliding chars + val topicWithCollidingChar = "test.a" + val createOpts = new TopicCommandOptions(Array("--partitions", "1", + "--replication-factor", "1", + "--topic", topicWithCollidingChar)) + createAndWaitTopic(createOpts) + + // delete the topic + val deleteOpts = new TopicCommandOptions(Array("--topic", topicWithCollidingChar)) + + if (!isKRaftTest()) { + val deletePath = DeleteTopicsTopicZNode.path(topicWithCollidingChar) + assertFalse(zkClient.pathExists(deletePath), "Delete path for topic shouldn't exist before deletion.") + } + topicService.deleteTopic(deleteOpts) + TestUtils.verifyTopicDeletion(zkClientOrNull, topicWithCollidingChar, 1, brokers) + + val createTopic: Executable = () => createAndWaitTopic(createOpts) + assertDoesNotThrow(createTopic) + } + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testDeleteInternalTopic(quorum: String): Unit = { diff --git a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala index 56bc47c79e956..23b56b8e91fd3 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala @@ -342,15 +342,16 @@ abstract class AbstractConsumerTest extends BaseRequestTest { protected class ConsumerAssignmentPoller(consumer: Consumer[Array[Byte], Array[Byte]], topicsToSubscribe: List[String], - partitionsToAssign: Set[TopicPartition]) + partitionsToAssign: Set[TopicPartition], + userRebalanceListener: ConsumerRebalanceListener) extends ShutdownableThread("daemon-consumer-assignment", false) { def this(consumer: Consumer[Array[Byte], Array[Byte]], topicsToSubscribe: List[String]) = { - this(consumer, topicsToSubscribe, Set.empty[TopicPartition]) + this(consumer, topicsToSubscribe, Set.empty[TopicPartition], null) } def this(consumer: Consumer[Array[Byte], Array[Byte]], partitionsToAssign: Set[TopicPartition]) = { - this(consumer, List.empty[String], partitionsToAssign) + this(consumer, List.empty[String], partitionsToAssign, null) } @volatile var thrownException: Option[Throwable] = None @@ -363,10 +364,14 @@ abstract class AbstractConsumerTest extends BaseRequestTest { val rebalanceListener: ConsumerRebalanceListener = new ConsumerRebalanceListener { override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]) = { partitionAssignment ++= partitions.toArray(new Array[TopicPartition](0)) + if (userRebalanceListener != null) + userRebalanceListener.onPartitionsAssigned(partitions) } override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]) = { partitionAssignment --= partitions.toArray(new Array[TopicPartition](0)) + if (userRebalanceListener != null) + userRebalanceListener.onPartitionsRevoked(partitions) } } diff --git a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala index 4d48bf5a8656b..5b2213a65e962 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala @@ -20,17 +20,19 @@ import kafka.log.LogConfig import kafka.server.{Defaults, KafkaConfig} import kafka.utils.TestUtils.assertFutureExceptionTypeEquals import kafka.utils.{Logging, TestInfoUtils, TestUtils} -import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigsOptions, Config, ConfigEntry} +import org.apache.kafka.clients.admin.AlterConfigOp.OpType +import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp, AlterConfigsOptions, Config, ConfigEntry} import org.apache.kafka.common.config.{ConfigResource, TopicConfig} import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, PolicyViolationException} import org.apache.kafka.common.utils.Utils import org.apache.kafka.server.policy.AlterConfigPolicy -import org.junit.jupiter.api.Assertions.{assertEquals, assertNull} +import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource import scala.annotation.nowarn +import scala.collection.mutable import scala.jdk.CollectionConverters._ /** @@ -121,6 +123,14 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with val topicResource3 = new ConfigResource(ConfigResource.Type.TOPIC, topic3) createTopic(topic3, 1, 1) + // Set a mutable broker config + val brokerResource = new ConfigResource(ConfigResource.Type.BROKER, brokers.head.config.brokerId.toString) + val brokerConfigs = Seq(new ConfigEntry(KafkaConfig.MessageMaxBytesProp, "50000")).asJava + val alterResult1 = client.alterConfigs(Map(brokerResource -> new Config(brokerConfigs)).asJava) + alterResult1.all.get + assertEquals(Set(KafkaConfig.MessageMaxBytesProp), validationsForResource(brokerResource).head.configs().keySet().asScala) + validations.clear() + val topicConfigEntries1 = Seq( new ConfigEntry(LogConfig.MinCleanableDirtyRatioProp, "0.9"), new ConfigEntry(LogConfig.MinInSyncReplicasProp, "2") // policy doesn't allow this @@ -130,7 +140,6 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with val topicConfigEntries3 = Seq(new ConfigEntry(LogConfig.MinInSyncReplicasProp, "-1")).asJava - val brokerResource = new ConfigResource(ConfigResource.Type.BROKER, brokers.head.config.brokerId.toString) val brokerConfigEntries = Seq(new ConfigEntry(KafkaConfig.SslTruststorePasswordProp, "12313")).asJava // Alter configs: second is valid, the others are invalid @@ -146,6 +155,9 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with alterResult.values.get(topicResource2).get assertFutureExceptionTypeEquals(alterResult.values.get(topicResource3), classOf[InvalidConfigurationException]) assertFutureExceptionTypeEquals(alterResult.values.get(brokerResource), classOf[InvalidRequestException]) + assertTrue(validationsForResource(brokerResource).isEmpty, + "Should not see the broker resource in the AlterConfig policy when the broker configs are not being updated.") + validations.clear() // Verify that the second resource was updated and the others were not ensureConsistentKRaftMetadata() @@ -175,6 +187,9 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with alterResult.values.get(topicResource2).get assertFutureExceptionTypeEquals(alterResult.values.get(topicResource3), classOf[InvalidConfigurationException]) assertFutureExceptionTypeEquals(alterResult.values.get(brokerResource), classOf[InvalidRequestException]) + assertTrue(validationsForResource(brokerResource).isEmpty, + "Should not see the broker resource in the AlterConfig policy when the broker configs are not being updated.") + validations.clear() // Verify that no resources are updated since validate_only = true ensureConsistentKRaftMetadata() @@ -188,27 +203,44 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with assertEquals("0.8", configs.get(topicResource2).get(LogConfig.MinCleanableDirtyRatioProp).value) assertNull(configs.get(brokerResource).get(KafkaConfig.SslTruststorePasswordProp).value) + + // Do an incremental alter config on the broker, ensure we don't see the broker config we set earlier in the policy + alterResult = client.incrementalAlterConfigs(Map( + brokerResource -> + Seq(new AlterConfigOp( + new ConfigEntry(KafkaConfig.MaxConnectionsProp, "9999"), OpType.SET) + ).asJavaCollection + ).asJava) + alterResult.all.get + assertEquals(Set(KafkaConfig.MaxConnectionsProp), validationsForResource(brokerResource).head.configs().keySet().asScala) } } object AdminClientWithPoliciesIntegrationTest { + val validations = new mutable.ListBuffer[AlterConfigPolicy.RequestMetadata]() + + def validationsForResource(resource: ConfigResource): Seq[AlterConfigPolicy.RequestMetadata] = { + validations.filter { req => req.resource().equals(resource) }.toSeq + } + class Policy extends AlterConfigPolicy { var configs: Map[String, _] = _ var closed = false def configure(configs: util.Map[String, _]): Unit = { + validations.clear() this.configs = configs.asScala.toMap } def validate(requestMetadata: AlterConfigPolicy.RequestMetadata): Unit = { + validations.append(requestMetadata) require(!closed, "Policy should not be closed") require(!configs.isEmpty, "configure should have been called with non empty configs") require(!requestMetadata.configs.isEmpty, "request configs should not be empty") require(requestMetadata.resource.name.nonEmpty, "resource name should not be empty") - require(requestMetadata.resource.name.contains("topic")) if (requestMetadata.configs.containsKey(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG)) throw new PolicyViolationException("Min in sync replicas cannot be updated") } diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index a109ae8ce4c64..ff1b2f5934de0 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -25,7 +25,7 @@ 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} +import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, NewTopic} import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.clients.producer._ @@ -2619,4 +2619,16 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ) } + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testPrefixAcls(quorum: String): Unit = { + addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, CREATE, ALLOW)), + new ResourcePattern(TOPIC, "f", PREFIXED)) + addAndVerifyAcls(Set(new AccessControlEntry("User:otherPrincipal", WildcardHost, CREATE, DENY)), + new ResourcePattern(TOPIC, "fooa", PREFIXED)) + addAndVerifyAcls(Set(new AccessControlEntry("User:otherPrincipal", WildcardHost, CREATE, ALLOW)), + new ResourcePattern(TOPIC, "foob", PREFIXED)) + createAdminClient().createTopics(Collections. + singletonList(new NewTopic("foobar", 1, 1.toShort))).all().get() + } } diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 61870b073d845..ce3cd32afdea3 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -19,22 +19,24 @@ package kafka.api import java.time.Duration import java.nio.charset.StandardCharsets -import java.util.Properties +import java.util.{Collections, Properties} import java.util.concurrent.TimeUnit - import kafka.integration.KafkaServerTestHarness import kafka.log.LogConfig import kafka.server.KafkaConfig -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} +import org.apache.kafka.clients.admin.{Admin, NewPartitions} import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer._ import org.apache.kafka.common.errors.TimeoutException -import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.network.{ListenerName, Mode} import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.{KafkaException, TopicPartition} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.jdk.CollectionConverters._ import scala.collection.mutable.Buffer @@ -42,16 +44,17 @@ import scala.concurrent.ExecutionException abstract class BaseProducerSendTest extends KafkaServerTestHarness { - def generateConfigs = { + def generateConfigs: scala.collection.Seq[KafkaConfig] = { val overridingProps = new Properties() val numServers = 2 overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) - TestUtils.createBrokerConfigs(numServers, zkConnect, false, interBrokerSecurityProtocol = Some(securityProtocol), + TestUtils.createBrokerConfigs(numServers, zkConnectOrNull, false, interBrokerSecurityProtocol = Some(securityProtocol), trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties).map(KafkaConfig.fromProps(_, overridingProps)) } private var consumer: KafkaConsumer[Array[Byte], Array[Byte]] = _ private val producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() + protected var admin: Admin = null protected val topic = "topic" private val numRecords = 100 @@ -59,6 +62,15 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { super.setUp(testInfo) + + admin = TestUtils.createAdminClient(brokers, listenerName, + TestUtils.securityConfigs(Mode.CLIENT, + securityProtocol, + trustStoreFile, + "adminClient", + TestUtils.SslCertificateCn, + clientSaslProperties)) + consumer = TestUtils.createConsumer( bootstrapServers(listenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)), securityProtocol = SecurityProtocol.PLAINTEXT @@ -70,6 +82,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { consumer.close() // Ensure that all producers are closed since unclosed producers impact other tests when Kafka server ports are reused producers.foreach(_.close()) + admin.close() super.tearDown() } @@ -105,8 +118,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { * 1. Send with null key/value/partition-id should be accepted; send with null topic should be rejected. * 2. Last message of the non-blocking send should return the correct offset metadata */ - @Test - def testSendOffset(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendOffset(quorum: String): Unit = { val producer = createProducer() val partition = 0 @@ -134,7 +148,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { try { // create topic - createTopic(topic, 1, 2) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2) // send a normal record val record0 = new ProducerRecord[Array[Byte], Array[Byte]](topic, partition, "key".getBytes(StandardCharsets.UTF_8), @@ -166,8 +180,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { } } - @Test - def testSendCompressedMessageWithCreateTime(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendCompressedMessageWithCreateTime(quorum: String): Unit = { val producer = createProducer( compressionType = "gzip", lingerMs = Int.MaxValue, @@ -175,8 +190,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { sendAndVerifyTimestamp(producer, TimestampType.CREATE_TIME) } - @Test - def testSendNonCompressedMessageWithCreateTime(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendNonCompressedMessageWithCreateTime(quorum: String): Unit = { val producer = createProducer(lingerMs = Int.MaxValue, deliveryTimeoutMs = Int.MaxValue) sendAndVerifyTimestamp(producer, TimestampType.CREATE_TIME) } @@ -186,7 +202,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { timeoutMs: Long = 20000L): Unit = { val partition = 0 try { - createTopic(topic, 1, 2) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2) val futures = for (i <- 1 to numRecords) yield { val record = new ProducerRecord(topic, partition, s"key$i".getBytes(StandardCharsets.UTF_8), @@ -241,7 +257,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { topicProps.setProperty(LogConfig.MessageTimestampTypeProp, "LogAppendTime") else topicProps.setProperty(LogConfig.MessageTimestampTypeProp, "CreateTime") - createTopic(topic, 1, 2, topicProps) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2, topicConfig = topicProps) val recordAndFutures = for (i <- 1 to numRecords) yield { val record = new ProducerRecord(topic, partition, baseTimestamp + i, s"key$i".getBytes(StandardCharsets.UTF_8), @@ -267,13 +283,14 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { * * After close() returns, all messages should be sent with correct returned offset metadata */ - @Test - def testClose(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testClose(quorum: String): Unit = { val producer = createProducer() try { // create topic - createTopic(topic, 1, 2) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2) // non-blocking send a list of records val record0 = new ProducerRecord[Array[Byte], Array[Byte]](topic, null, "key".getBytes(StandardCharsets.UTF_8), @@ -300,12 +317,13 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { * * The specified partition-id should be respected */ - @Test - def testSendToPartition(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendToPartition(quorum: String): Unit = { val producer = createProducer() try { - createTopic(topic, 2, 2) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 2, 2) val partition = 1 val now = System.currentTimeMillis() @@ -345,14 +363,15 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { * Producer will attempt to send messages to the partition specified in each record, and should * succeed as long as the partition is included in the metadata. */ - @Test - def testSendBeforeAndAfterPartitionExpansion(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendBeforeAndAfterPartitionExpansion(quorum: String): Unit = { val producer = createProducer(maxBlockMs = 5 * 1000L) // create topic - createTopic(topic, 1, 2) - val partition0 = 0 + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2) + val partition0 = 0 var futures0 = (1 to numRecords).map { i => producer.send(new ProducerRecord(topic, partition0, null, ("value" + i).getBytes(StandardCharsets.UTF_8))) }.map(_.get(30, TimeUnit.SECONDS)) @@ -369,13 +388,11 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { val e = assertThrows(classOf[ExecutionException], () => producer.send(new ProducerRecord(topic, partition1, null, "value".getBytes(StandardCharsets.UTF_8))).get()) assertEquals(classOf[TimeoutException], e.getCause.getClass) - val existingAssignment = zkClient.getFullReplicaAssignmentForTopics(Set(topic)).map { - case (topicPartition, assignment) => topicPartition.partition -> assignment - } - adminZkClient.addPartitions(topic, existingAssignment, adminZkClient.getBrokerMetadatas(), 2) + admin.createPartitions(Collections.singletonMap(topic, NewPartitions.increaseTo(2))).all().get() + // read metadata from a broker and verify the new topic partitions exist - TestUtils.waitForPartitionMetadata(servers, topic, 0) - TestUtils.waitForPartitionMetadata(servers, topic, 1) + TestUtils.waitForPartitionMetadata(brokers, topic, 0) + TestUtils.waitForPartitionMetadata(brokers, topic, 1) // send records to the newly added partition after confirming that metadata have been updated. val futures1 = (1 to numRecords).map { i => @@ -404,11 +421,12 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { /** * Test that flush immediately sends all accumulated requests. */ - @Test - def testFlush(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testFlush(quorum: String): Unit = { val producer = createProducer(lingerMs = Int.MaxValue, deliveryTimeoutMs = Int.MaxValue) try { - createTopic(topic, 2, 2) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 2, 2) val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, "value".getBytes(StandardCharsets.UTF_8)) for (_ <- 0 until 50) { @@ -425,9 +443,10 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { /** * Test close with zero timeout from caller thread */ - @Test - def testCloseWithZeroTimeoutFromCallerThread(): Unit = { - createTopic(topic, 2, 2) + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCloseWithZeroTimeoutFromCallerThread(quorum: String): Unit = { + TestUtils.createTopicWithAdmin(admin, topic, brokers, 2, 2) val partition = 0 consumer.assign(List(new TopicPartition(topic, partition)).asJava) val record0 = new ProducerRecord[Array[Byte], Array[Byte]](topic, partition, null, @@ -450,9 +469,10 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { /** * Test close with zero and non-zero timeout from sender thread */ - @Test - def testCloseWithZeroTimeoutFromSenderThread(): Unit = { - createTopic(topic, 1, 2) + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCloseWithZeroTimeoutFromSenderThread(quorum: String): Unit = { + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2) val partition = 0 consumer.assign(List(new TopicPartition(topic, partition)).asJava) val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, partition, null, "value".getBytes(StandardCharsets.UTF_8)) diff --git a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala index 40d4cef7f82ee..457a1ddd48d7d 100644 --- a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala @@ -173,7 +173,8 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { consumer.subscribe(Collections.singleton(topic1)) val endTimeMs = System.currentTimeMillis + 10000 var throttled = false - while ((!throttled || quotaTestClients.exemptRequestMetric == null) && System.currentTimeMillis < endTimeMs) { + while ((!throttled || quotaTestClients.exemptRequestMetric == null || metricValue(quotaTestClients.exemptRequestMetric) <= 0) + && System.currentTimeMillis < endTimeMs) { consumer.poll(Duration.ofMillis(100L)) val throttleMetric = quotaTestClients.throttleMetric(QuotaType.Request, consumerClientId) throttled = throttleMetric != null && metricValue(throttleMetric) > 0 diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 203c04a68a7a9..1656af08bc15e 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -33,7 +33,6 @@ import kafka.server.{Defaults, DynamicConfig, KafkaConfig} import kafka.utils.TestUtils._ import kafka.utils.{Log4jController, TestInfoUtils, TestUtils} import org.apache.kafka.clients.HostResolver -import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.clients.admin.ConfigEntry.ConfigSource import org.apache.kafka.clients.admin._ import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} @@ -159,22 +158,6 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { waitForTopics(client, List(), topics) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk")) // KRaft mode will be supported in KAFKA-13910 - def testMetadataRefresh(quorum: String): Unit = { - client = Admin.create(createConfig) - val topics = Seq("mytopic") - val newTopics = Seq(new NewTopic("mytopic", 3, 3.toShort)) - client.createTopics(newTopics.asJava).all.get() - waitForTopics(client, expectedPresent = topics, expectedMissing = List()) - - val controller = brokers.find(_.config.brokerId == brokers.flatMap(_.metadataCache.getControllerId).head).get - controller.shutdown() - controller.awaitShutdown() - val topicDesc = client.describeTopics(topics.asJava).allTopicNames.get() - assertEquals(topics.toSet, topicDesc.keySet.asScala) - } - /** * describe should not auto create topics */ @@ -821,10 +804,10 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testReplicaCanFetchFromLogStartOffsetAfterDeleteRecords(quorum: String): Unit = { val leaders = createTopic(topic, replicationFactor = brokerCount) - val followerIndex = if (leaders(0) != brokers(0).config.brokerId) 0 else 1 + val followerIndex = if (leaders(0) != brokers.head.config.brokerId) 0 else 1 def waitForFollowerLog(expectedStartOffset: Long, expectedEndOffset: Long): Unit = { - TestUtils.waitUntilTrue(() => brokers(followerIndex).replicaManager.localLog(topicPartition) != None, + TestUtils.waitUntilTrue(() => brokers(followerIndex).replicaManager.localLog(topicPartition).isDefined, "Expected follower to create replica for partition") // wait until the follower discovers that log start offset moved beyond its HW @@ -862,6 +845,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val result1 = client.deleteRecords(Map(topicPartition -> RecordsToDelete.beforeOffset(117L)).asJava) result1.all().get() restartDeadBrokers() + TestUtils.waitForBrokersInIsr(client, topicPartition, Set(followerIndex)) waitForFollowerLog(expectedStartOffset=117L, expectedEndOffset=200L) } @@ -1522,7 +1506,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { // Now change the preferred leader to 1 changePreferredLeader(prefer1) // but shut it down... - brokers(1).shutdown() + killBroker(1) TestUtils.waitForBrokersOutOfIsr(client, Set(partition1, partition2), Set(1)) def assertPreferredLeaderNotAvailable( @@ -1576,9 +1560,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { TestUtils.assertLeader(client, partition1, broker1) - brokers(broker2).shutdown() + killBroker(broker2) TestUtils.waitForBrokersOutOfIsr(client, Set(partition1), Set(broker2)) - brokers(broker1).shutdown() + killBroker(broker1) TestUtils.assertNoLeader(client, partition1) brokers(broker2).startup() @@ -1610,9 +1594,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { TestUtils.assertLeader(client, partition1, broker1) TestUtils.assertLeader(client, partition2, broker1) - brokers(broker2).shutdown() + killBroker(broker2) TestUtils.waitForBrokersOutOfIsr(client, Set(partition1, partition2), Set(broker2)) - brokers(broker1).shutdown() + killBroker(broker1) TestUtils.assertNoLeader(client, partition1) TestUtils.assertNoLeader(client, partition2) brokers(broker2).startup() @@ -1648,9 +1632,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { TestUtils.assertLeader(client, partition1, broker1) TestUtils.assertLeader(client, partition2, broker1) - brokers(broker2).shutdown() + killBroker(broker2) TestUtils.waitForBrokersOutOfIsr(client, Set(partition1), Set(broker2)) - brokers(broker1).shutdown() + killBroker(broker1) TestUtils.assertNoLeader(client, partition1) TestUtils.assertLeader(client, partition2, broker3) brokers(broker2).startup() @@ -1708,9 +1692,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { TestUtils.assertLeader(client, partition1, broker1) - brokers(broker2).shutdown() + killBroker(broker2) TestUtils.waitForBrokersOutOfIsr(client, Set(partition1), Set(broker2)) - brokers(broker1).shutdown() + killBroker(broker1) TestUtils.assertNoLeader(client, partition1) val electResult = client.electLeaders(ElectionType.UNCLEAN, Set(partition1).asJava) @@ -1737,7 +1721,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { TestUtils.assertLeader(client, partition1, broker1) - brokers(broker1).shutdown() + killBroker(broker1) TestUtils.assertLeader(client, partition1, broker2) brokers(broker1).startup() @@ -1769,9 +1753,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { TestUtils.assertLeader(client, partition1, broker1) TestUtils.assertLeader(client, partition2, broker1) - brokers(broker2).shutdown() + killBroker(broker2) TestUtils.waitForBrokersOutOfIsr(client, Set(partition1), Set(broker2)) - brokers(broker1).shutdown() + killBroker(broker1) TestUtils.assertNoLeader(client, partition1) TestUtils.assertLeader(client, partition2, broker3) brokers(broker2).startup() @@ -2480,6 +2464,38 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { } } + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testAppendConfigToEmptyDefaultValue(ignored: String): Unit = { + testAppendConfig(new Properties(), "0:0", "0:0") + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testAppendConfigToExistentValue(ignored: String): Unit = { + val props = new Properties(); + props.setProperty(LogConfig.LeaderReplicationThrottledReplicasProp, "1:1") + testAppendConfig(props, "0:0", "1:1,0:0") + } + + private def testAppendConfig(props: Properties, append: String, expected: String): Unit = { + client = Admin.create(createConfig) + createTopic(topic, topicConfig = props) + val topicResource = new ConfigResource(ConfigResource.Type.TOPIC, topic) + val topicAlterConfigs = Seq( + new AlterConfigOp(new ConfigEntry(LogConfig.LeaderReplicationThrottledReplicasProp, append), AlterConfigOp.OpType.APPEND), + ).asJavaCollection + + val alterResult = client.incrementalAlterConfigs(Map( + topicResource -> topicAlterConfigs + ).asJava) + alterResult.all().get(15, TimeUnit.SECONDS) + + ensureConsistentKRaftMetadata() + val config = client.describeConfigs(List(topicResource).asJava).all().get().get(topicResource).get(LogConfig.LeaderReplicationThrottledReplicasProp) + assertEquals(expected, config.value()) + } + /** * Test that createTopics returns the dynamic configurations of the topics that were created. * @@ -2491,19 +2507,29 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { def testCreateTopicsReturnsConfigs(quorum: String): Unit = { client = Admin.create(super.createConfig) - val alterMap = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]] - alterMap.put(new ConfigResource(ConfigResource.Type.BROKER, ""), util.Arrays.asList( - new AlterConfigOp(new ConfigEntry(KafkaConfig.LogRetentionTimeMillisProp, "10800000"), OpType.SET))) - (brokers.map(_.config) ++ controllerServers.map(_.config)).foreach { case config => - alterMap.put(new ConfigResource(ConfigResource.Type.BROKER, config.nodeId.toString()), - util.Arrays.asList(new AlterConfigOp(new ConfigEntry( - KafkaConfig.LogCleanerDeleteRetentionMsProp, "34"), OpType.SET))) + val newLogRetentionProperties = new Properties + newLogRetentionProperties.put(KafkaConfig.LogRetentionTimeMillisProp, "10800000") + TestUtils.incrementalAlterConfigs(null, client, newLogRetentionProperties, perBrokerConfig = false) + .all().get(15, TimeUnit.SECONDS) + + val newLogCleanerDeleteRetention = new Properties + newLogCleanerDeleteRetention.put(KafkaConfig.LogCleanerDeleteRetentionMsProp, "34") + TestUtils.incrementalAlterConfigs(brokers, client, newLogCleanerDeleteRetention, perBrokerConfig = true) + .all().get(15, TimeUnit.SECONDS) + + if (isKRaftTest()) { + ensureConsistentKRaftMetadata() + } else { + waitUntilTrue(() => brokers.forall(_.config.originals.getOrDefault( + KafkaConfig.LogCleanerDeleteRetentionMsProp, "").toString.equals("34")), + s"Timed out waiting for change to ${KafkaConfig.LogCleanerDeleteRetentionMsProp}", + waitTimeMs = 60000L) + + waitUntilTrue(() => brokers.forall(_.config.originals.getOrDefault( + KafkaConfig.LogRetentionTimeMillisProp, "").toString.equals("10800000")), + s"Timed out waiting for change to ${KafkaConfig.LogRetentionTimeMillisProp}", + waitTimeMs = 60000L) } - client.incrementalAlterConfigs(alterMap).all().get() - waitUntilTrue(() => brokers.forall(_.config.originals.getOrDefault( - KafkaConfig.LogCleanerDeleteRetentionMsProp, "").toString.equals("34")), - s"Timed out waiting for change to ${KafkaConfig.LogCleanerDeleteRetentionMsProp}", - waitTimeMs = 60000L) val newTopics = Seq(new NewTopic("foo", Map((0: Integer) -> Seq[Integer](1, 2).asJava, (1: Integer) -> Seq[Integer](2, 0).asJava).asJava). diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 4ede241b0c57a..92670dba3b3b0 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -37,7 +37,11 @@ import kafka.server.QuotaType import kafka.server.KafkaServer import org.apache.kafka.clients.admin.NewPartitions import org.apache.kafka.clients.admin.NewTopic +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource +import java.util.concurrent.TimeUnit +import java.util.concurrent.locks.ReentrantLock import scala.collection.mutable /* We have some tests in this class instead of `BaseConsumerTest` in order to keep the build time under control. */ @@ -633,7 +637,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(0L, consumer.position(tp), "position() on a partition that we are subscribed to should reset the offset") consumer.commitSync() assertEquals(0L, consumer.committed(Set(tp).asJava).get(tp).offset) - consumeAndVerifyRecords(consumer = consumer, numRecords = 5, startingOffset = 0, startingTimestamp = startingTimestamp) assertEquals(5L, consumer.position(tp), "After consuming 5 records, position should be 5") consumer.commitSync() @@ -969,6 +972,89 @@ class PlaintextConsumerTest extends BaseConsumerTest { } } + @ParameterizedTest + @ValueSource(strings = Array( + "org.apache.kafka.clients.consumer.CooperativeStickyAssignor", + "org.apache.kafka.clients.consumer.RangeAssignor")) + def testRebalanceAndRejoin(assignmentStrategy: String): Unit = { + // create 2 consumers + 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 @@ -1555,12 +1641,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { servers.foreach(assertNoMetric(_, "throttle-time", QuotaType.Request, producerClientId)) servers.foreach(assertNoMetric(_, "request-time", QuotaType.Request, consumerClientId)) servers.foreach(assertNoMetric(_, "throttle-time", QuotaType.Request, consumerClientId)) - - def assertNoExemptRequestMetric(broker: KafkaServer): Unit = { - val metricName = broker.metrics.metricName("exempt-request-time", QuotaType.Request.toString, "") - assertNull(broker.metrics.metric(metricName), "Metric should not have been created " + metricName) - } - servers.foreach(assertNoExemptRequestMetric) } def runMultiConsumerSessionTimeoutTest(closeConsumer: Boolean): Unit = { diff --git a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala index 06ff201e0b2e6..21d9a8870d13c 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala @@ -19,22 +19,26 @@ package kafka.api import java.util.Properties import java.util.concurrent.{ExecutionException, Future, TimeUnit} - import kafka.log.LogConfig import kafka.server.Defaults -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.producer.{BufferExhaustedException, KafkaProducer, ProducerConfig, ProducerRecord, RecordMetadata} import org.apache.kafka.common.errors.{InvalidTimestampException, RecordTooLargeException, SerializationException, TimeoutException} import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch, Records, TimestampType} import org.apache.kafka.common.serialization.ByteArraySerializer import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test +import org.junit.jupiter.api.Timeout +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource + +import java.nio.charset.StandardCharsets class PlaintextProducerSendTest extends BaseProducerSendTest { - @Test - def testWrongSerializer(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testWrongSerializer(quorum: String): Unit = { val producerProps = new Properties() producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()) producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") @@ -44,8 +48,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { assertThrows(classOf[SerializationException], () => producer.send(record)) } - @Test - def testBatchSizeZero(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testBatchSizeZero(quorum: String): Unit = { val producer = createProducer( lingerMs = Int.MaxValue, deliveryTimeoutMs = Int.MaxValue, @@ -53,8 +58,33 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { sendAndVerify(producer) } - @Test - def testSendCompressedMessageWithLogAppendTime(): Unit = { + @Timeout(value = 15, unit = TimeUnit.SECONDS) + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testBatchSizeZeroNoPartitionNoRecordKey(quorum: String): Unit = { + val producer = createProducer(batchSize = 0) + val numRecords = 10; + try { + TestUtils.createTopicWithAdmin(admin, topic, brokers, 2) + val futures = for (i <- 1 to numRecords) yield { + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, null, s"value$i".getBytes(StandardCharsets.UTF_8)) + producer.send(record) + } + producer.flush() + val lastOffset = futures.foldLeft(0) { (offset, future) => + val recordMetadata = future.get + assertEquals(topic, recordMetadata.topic) + offset + 1 + } + assertEquals(numRecords, lastOffset) + } finally { + producer.close() + } + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendCompressedMessageWithLogAppendTime(quorum: String): Unit = { val producer = createProducer( compressionType = "gzip", lingerMs = Int.MaxValue, @@ -62,8 +92,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { sendAndVerifyTimestamp(producer, TimestampType.LOG_APPEND_TIME) } - @Test - def testSendNonCompressedMessageWithLogAppendTime(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendNonCompressedMessageWithLogAppendTime(quorum: String): Unit = { val producer = createProducer(lingerMs = Int.MaxValue, deliveryTimeoutMs = Int.MaxValue) sendAndVerifyTimestamp(producer, TimestampType.LOG_APPEND_TIME) } @@ -73,8 +104,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { * * The topic should be created upon sending the first message */ - @Test - def testAutoCreateTopic(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testAutoCreateTopic(quorum: String): Unit = { val producer = createProducer() try { // Send a message to auto-create the topic @@ -82,18 +114,18 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { assertEquals(0L, producer.send(record).get.offset, "Should have offset 0") // double check that the topic is created with leader elected - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) - + TestUtils.waitUntilLeaderIsElectedOrChangedWithAdmin(admin, topic, 0) } finally { producer.close() } } - @Test - def testSendWithInvalidCreateTime(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendWithInvalidCreateTime(quorum: String): Unit = { val topicProps = new Properties() topicProps.setProperty(LogConfig.MessageTimestampDifferenceMaxMsProp, "1000") - createTopic(topic, 1, 2, topicProps) + TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2, topicConfig = topicProps) val producer = createProducer() try { @@ -118,8 +150,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { // Test that producer with max.block.ms=0 can be used to send in non-blocking mode // where requests are failed immediately without blocking if metadata is not available // or buffer is full. - @Test - def testNonBlockingProducer(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testNonBlockingProducer(quorum: String): Unit = { def send(producer: KafkaProducer[Array[Byte],Array[Byte]]): Future[RecordMetadata] = { producer.send(new ProducerRecord(topic, 0, "key".getBytes, new Array[Byte](1000))) @@ -173,8 +206,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { verifySendSuccess(future2) // previous batch should be completed and sent now } - @Test - def testSendRecordBatchWithMaxRequestSizeAndHigher(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testSendRecordBatchWithMaxRequestSizeAndHigher(quorum: String): Unit = { val producerProps = new Properties() producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()) val producer = registerProducer(new KafkaProducer(producerProps, new ByteArraySerializer, new ByteArraySerializer)) diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index ccdfe7d3d3676..07d9ccb024f9a 100755 --- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -17,19 +17,19 @@ package kafka.api.test -import kafka.server.{KafkaConfig, KafkaServer} +import kafka.server.{KafkaBroker, KafkaConfig, QuorumTestHarness} import kafka.utils.TestUtils -import kafka.server.QuorumTestHarness import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.serialization.ByteArraySerializer import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{Arguments, MethodSource} +import org.junit.jupiter.params.provider.CsvSource import java.util.{Collections, Properties} -import scala.jdk.CollectionConverters._ class ProducerCompressionTest extends QuorumTestHarness { @@ -37,18 +37,18 @@ class ProducerCompressionTest extends QuorumTestHarness { private val topic = "topic" private val numRecords = 2000 - private var server: KafkaServer = null + private var broker: KafkaBroker = null @BeforeEach override def setUp(testInfo: TestInfo): Unit = { super.setUp(testInfo) - val props = TestUtils.createBrokerConfig(brokerId, zkConnect) - server = TestUtils.createServer(KafkaConfig.fromProps(props)) + val props = TestUtils.createBrokerConfig(brokerId, zkConnectOrNull) + broker = createBroker(new KafkaConfig(props)) } @AfterEach override def tearDown(): Unit = { - TestUtils.shutdownServers(Seq(server)) + TestUtils.shutdownServers(Seq(broker)) super.tearDown() } @@ -58,11 +58,18 @@ class ProducerCompressionTest extends QuorumTestHarness { * Compressed messages should be able to sent and consumed correctly */ @ParameterizedTest - @MethodSource(Array("parameters")) - def testCompression(compression: String): Unit = { + @CsvSource(value = Array( + "kraft,none", + "kraft,gzip", + "kraft,snappy", + "kraft,lz4", + "kraft,zstd", + "zk,gzip" + )) + def testCompression(quorum: String, compression: String): Unit = { val producerProps = new Properties() - val bootstrapServers = TestUtils.plaintextBootstrapServers(Seq(server)) + val bootstrapServers = TestUtils.plaintextBootstrapServers(Seq(broker)) producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers) producerProps.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, compression) producerProps.put(ProducerConfig.BATCH_SIZE_CONFIG, "66000") @@ -72,7 +79,13 @@ class ProducerCompressionTest extends QuorumTestHarness { try { // create topic - TestUtils.createTopic(zkClient, topic, 1, 1, List(server)) + val admin = TestUtils.createAdminClient(Seq(broker), + ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)) + try { + TestUtils.createTopicWithAdmin(admin, topic, Seq(broker)) + } finally { + admin.close() + } val partition = 0 // prepare the messages @@ -103,15 +116,3 @@ class ProducerCompressionTest extends QuorumTestHarness { } } } - -object ProducerCompressionTest { - def parameters: java.util.stream.Stream[Arguments] = { - Seq( - Arguments.of("none"), - Arguments.of("gzip"), - Arguments.of("snappy"), - Arguments.of("lz4"), - Arguments.of("zstd") - ).asJava.stream() - } -} diff --git a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala index eacc58e76cc59..5dd82b6b224bc 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala @@ -18,15 +18,16 @@ package kafka.api import java.util.Properties - import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} import kafka.utils.TestUtils.consumeRecords import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.KafkaProducer import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.collection.Seq import scala.collection.mutable.Buffer @@ -37,7 +38,7 @@ import scala.jdk.CollectionConverters._ * A single broker is used to verify edge cases where different requests are queued on the same connection. */ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { - val numServers = 1 + val numBrokers = 1 val topic1 = "topic1" val topic2 = "topic2" @@ -47,7 +48,7 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { val transactionalConsumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() override def generateConfigs: Seq[KafkaConfig] = { - TestUtils.createBrokerConfigs(numServers, zkConnect).map(KafkaConfig.fromProps(_, serverProps())) + TestUtils.createBrokerConfigs(numBrokers, zkConnectOrNull).map(KafkaConfig.fromProps(_, serverProps())) } @BeforeEach @@ -55,8 +56,8 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { super.setUp(testInfo) val topicConfig = new Properties() topicConfig.put(KafkaConfig.MinInSyncReplicasProp, 1.toString) - createTopic(topic1, numPartitions, numServers, topicConfig) - createTopic(topic2, numPartitions, numServers, topicConfig) + createTopic(topic1, numPartitions, numBrokers, topicConfig) + createTopic(topic2, numPartitions, numBrokers, topicConfig) createTransactionalProducer("transactional-producer") createReadCommittedConsumer("transactional-group") @@ -69,10 +70,11 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { super.tearDown() } - @Test - def testTransactionalProducerSingleBrokerMaxInFlightOne(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testTransactionalProducerSingleBrokerMaxInFlightOne(quorum: String): Unit = { // We want to test with one broker to verify multiple requests queued on a connection - assertEquals(1, servers.size) + assertEquals(1, brokers.size) val producer = transactionalProducers.head val consumer = transactionalConsumers.head @@ -124,7 +126,7 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { } private def createTransactionalProducer(transactionalId: String): KafkaProducer[Array[Byte], Array[Byte]] = { - val producer = TestUtils.createTransactionalProducer(transactionalId, servers, maxInFlight = 1) + val producer = TestUtils.createTransactionalProducer(transactionalId, brokers, maxInFlight = 1) transactionalProducers += producer producer } diff --git a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala index 7d3203e93095e..558f0041e0af6 100644 --- a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala @@ -46,7 +46,7 @@ class ProducerIdsIntegrationTest { @ClusterTests(Array( new ClusterTest(clusterType = Type.ZK, brokers = 3, metadataVersion = MetadataVersion.IBP_2_8_IV1), new ClusterTest(clusterType = Type.ZK, brokers = 3, metadataVersion = MetadataVersion.IBP_3_0_IV0), - new ClusterTest(clusterType = Type.KRAFT, brokers = 3, metadataVersion = MetadataVersion.IBP_3_0_IV1) + new ClusterTest(clusterType = Type.KRAFT, brokers = 3, metadataVersion = MetadataVersion.IBP_3_3_IV0) )) def testUniqueProducerIds(clusterInstance: ClusterInstance): Unit = { verifyUniqueIds(clusterInstance) diff --git a/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala b/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala index f8893bd1dab4c..b9f05d5aa8738 100644 --- a/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala +++ b/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package integration.kafka.network +package kafka.network import kafka.server.{BaseRequestTest, Defaults, KafkaConfig} import kafka.utils.TestUtils diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index ccfe63e7b566d..295ad061211b8 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -31,7 +31,7 @@ import com.yammer.metrics.core.MetricName import kafka.admin.ConfigCommand import kafka.api.{KafkaSasl, SaslSetup} import kafka.controller.{ControllerBrokerStateInfo, ControllerChannelManager} -import kafka.log.{CleanerConfig, LogConfig} +import kafka.log.{CleanerConfig, LogConfig, UnifiedLog} import kafka.message.ProducerCompressionCodec import kafka.network.{Processor, RequestChannel} import kafka.utils._ @@ -64,7 +64,10 @@ import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Test, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource +import java.util.concurrent.atomic.AtomicInteger import scala.annotation.nowarn import scala.collection._ import scala.collection.mutable.ArrayBuffer @@ -80,7 +83,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup import DynamicBrokerReconfigurationTest._ - private val servers = new ArrayBuffer[KafkaServer] + private val servers = new ArrayBuffer[KafkaBroker] private val numServers = 3 private val numPartitions = 10 private val producers = new ArrayBuffer[KafkaProducer[String, String]] @@ -111,15 +114,22 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup (0 until numServers).foreach { brokerId => - val props = TestUtils.createBrokerConfig(brokerId, zkConnect) + val props = if (isKRaftTest()) { + val properties = TestUtils.createBrokerConfig(brokerId, null) + properties.put(KafkaConfig.AdvertisedListenersProp, s"$SecureInternal://localhost:0, $SecureExternal://localhost:0") + properties + } else { + val properties = TestUtils.createBrokerConfig(brokerId, zkConnect) + properties.put(KafkaConfig.ZkEnableSecureAclsProp, "true") + properties + } props ++= securityProps(sslProperties1, TRUSTSTORE_PROPS) // Ensure that we can support multiple listeners per security protocol and multiple security protocols props.put(KafkaConfig.ListenersProp, s"$SecureInternal://localhost:0, $SecureExternal://localhost:0") - props.put(KafkaConfig.ListenerSecurityProtocolMapProp, s"$SecureInternal:SSL, $SecureExternal:SASL_SSL") + props.put(KafkaConfig.ListenerSecurityProtocolMapProp, s"$SecureInternal:SSL, $SecureExternal:SASL_SSL, CONTROLLER:$controllerListenerSecurityProtocol") props.put(KafkaConfig.InterBrokerListenerNameProp, SecureInternal) props.put(KafkaConfig.SslClientAuthProp, "requested") props.put(KafkaConfig.SaslMechanismInterBrokerProtocolProp, "PLAIN") - props.put(KafkaConfig.ZkEnableSecureAclsProp, "true") props.put(KafkaConfig.SaslEnabledMechanismsProp, kafkaServerSaslMechanisms.mkString(",")) props.put(KafkaConfig.LogSegmentBytesProp, "2000") // low value to test log rolling on config update props.put(KafkaConfig.NumReplicaFetchersProp, "2") // greater than one to test reducing threads @@ -138,17 +148,21 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props ++= securityProps(sslProperties1, KEYSTORE_PROPS, listenerPrefix(SecureExternal)) val kafkaConfig = KafkaConfig.fromProps(props) - configureDynamicKeystoreInZooKeeper(kafkaConfig, sslProperties1) + if (!isKRaftTest()) { + configureDynamicKeystoreInZooKeeper(kafkaConfig, sslProperties1) + } - servers += TestUtils.createServer(kafkaConfig) + servers += createBroker(kafkaConfig) } - TestUtils.createTopic(zkClient, topic, numPartitions, replicationFactor = numServers, servers) - TestUtils.createTopic(zkClient, Topic.GROUP_METADATA_TOPIC_NAME, servers.head.config.offsetsTopicPartitions, - replicationFactor = numServers, servers, servers.head.groupCoordinator.offsetsTopicConfigs) - createAdminClient(SecurityProtocol.SSL, SecureInternal) + TestUtils.createTopicWithAdmin(adminClients.head, topic, servers, numPartitions, replicationFactor = numServers) + TestUtils.createTopicWithAdmin(adminClients.head, Topic.GROUP_METADATA_TOPIC_NAME, servers, + numPartitions = servers.head.config.offsetsTopicPartitions, + replicationFactor = numServers, + topicConfig = servers.head.groupCoordinator.offsetsTopicConfigs) + TestMetricsReporter.testReporters.clear() } @@ -166,8 +180,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup closeSasl() } - @Test - def testConfigDescribeUsingAdminClient(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testConfigDescribeUsingAdminClient(quorum: String): Unit = { def verifyConfig(configName: String, configEntry: ConfigEntry, isSensitive: Boolean, isReadOnly: Boolean, expectedProps: Properties): Unit = { @@ -226,9 +241,12 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup val adminClient = adminClients.head alterSslKeystoreUsingConfigCommand(sslProperties1, SecureExternal) - val configDesc = describeConfig(adminClient) - verifySslConfig("listener.name.external.", sslProperties1, configDesc) - verifySslConfig("", invalidSslProperties, configDesc) + val configDesc = TestUtils.tryUntilNoAssertionError() { + val describeConfigsResult = describeConfig(adminClient) + verifySslConfig("listener.name.external.", sslProperties1, describeConfigsResult) + verifySslConfig("", invalidSslProperties, describeConfigsResult) + describeConfigsResult + } // Verify a few log configs with and without synonyms val expectedProps = new Properties @@ -262,8 +280,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup assertEquals(List((KafkaConfig.LogCleanerThreadsProp, ConfigSource.DEFAULT_CONFIG)), synonymsList(logCleanerThreads)) } - @Test - def testUpdatesUsingConfigProvider(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testUpdatesUsingConfigProvider(quorum: String): Unit = { val PollingIntervalVal = f"$${file:polling.interval:interval}" val PollingIntervalUpdateVal = f"$${file:polling.interval:updinterval}" val SslTruststoreTypeVal = f"$${file:ssl.truststore.type:storetype}" @@ -309,11 +328,13 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup assertFalse(reporter.kafkaMetrics.isEmpty, "No metrics found") } - // fetch from ZK, values should be unresolved - val props = fetchBrokerConfigsFromZooKeeper(servers.head) - assertTrue(props.getProperty(TestMetricsReporter.PollingIntervalProp) == PollingIntervalVal, "polling interval is not updated in ZK") - assertTrue(props.getProperty(configPrefix+KafkaConfig.SslTruststoreTypeProp) == SslTruststoreTypeVal, "store type is not updated in ZK") - assertTrue(props.getProperty(configPrefix+KafkaConfig.SslKeystorePasswordProp) == SslKeystorePasswordVal, "keystore password is not updated in ZK") + if (!isKRaftTest()) { + // fetch from ZK, values should be unresolved + val props = fetchBrokerConfigsFromZooKeeper(servers.head) + assertTrue(props.getProperty(TestMetricsReporter.PollingIntervalProp) == PollingIntervalVal, "polling interval is not updated in ZK") + assertTrue(props.getProperty(configPrefix + KafkaConfig.SslTruststoreTypeProp) == SslTruststoreTypeVal, "store type is not updated in ZK") + assertTrue(props.getProperty(configPrefix + KafkaConfig.SslKeystorePasswordProp) == SslKeystorePasswordVal, "keystore password is not updated in ZK") + } // verify the update // 1. verify update not occurring if the value of property is same. @@ -332,10 +353,11 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } } - @Test - def testKeyStoreAlter(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testKeyStoreAlter(quorum: String): Unit = { val topic2 = "testtopic2" - TestUtils.createTopic(zkClient, topic2, numPartitions, replicationFactor = numServers, servers) + TestUtils.createTopicWithAdmin(adminClients.head, topic2, servers, numPartitions, replicationFactor = numServers) // Start a producer and consumer that work with the current broker keystore. // This should continue working while changes are made @@ -355,7 +377,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup // Produce/consume should work with new truststore with new producer/consumer val producer = ProducerBuilder().trustStoreProps(sslProperties2).maxRetries(0).build() - val consumer = ConsumerBuilder("group1").trustStoreProps(sslProperties2).topic(topic2).build() + // Start the new consumer in a separate group than the continous consumer started at the beginning of the test so + // that it is not disrupted by rebalance. + val consumer = ConsumerBuilder("group2").trustStoreProps(sslProperties2).topic(topic2).build() verifyProduceConsume(producer, consumer, 10, topic2) // Broker keystore update for internal listener with incompatible keystore should fail without update @@ -399,8 +423,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup stopAndVerifyProduceConsume(producerThread, consumerThread) } - @Test - def testTrustStoreAlter(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testTrustStoreAlter(quorum: String): Unit = { val producerBuilder = ProducerBuilder().listenerName(SecureInternal).securityProtocol(SecurityProtocol.SSL) // Producer with new keystore should fail to connect before truststore update @@ -447,9 +472,12 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup assertFalse(response.wasDisconnected(), "Request failed because broker is not available") } + val group_id = new AtomicInteger(1) + def next_group_name(): String = s"alter-truststore-${group_id.getAndIncrement()}" + // Produce/consume should work with old as well as new client keystore - verifySslProduceConsume(sslProperties1, "alter-truststore-1") - verifySslProduceConsume(sslProperties2, "alter-truststore-2") + verifySslProduceConsume(sslProperties1, next_group_name()) + verifySslProduceConsume(sslProperties2, next_group_name()) // Revert to old truststore with only one certificate and update. Clients should connect only with old keystore. val oldTruststoreProps = new Properties @@ -458,7 +486,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup reconfigureServers(oldTruststoreProps, perBrokerConfig = true, (s"$prefix$SSL_TRUSTSTORE_LOCATION_CONFIG", sslProperties1.getProperty(SSL_TRUSTSTORE_LOCATION_CONFIG))) verifyAuthenticationFailure(producerBuilder.keyStoreProps(sslProperties2).build()) - verifySslProduceConsume(sslProperties1, "alter-truststore-3") + verifySslProduceConsume(sslProperties1, next_group_name()) // Update same truststore file to contain both certificates without changing any configs. // Clients should connect successfully with either keystore after admin client AlterConfigsRequest completes. @@ -466,8 +494,14 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup Paths.get(sslProperties1.getProperty(SSL_TRUSTSTORE_LOCATION_CONFIG)), StandardCopyOption.REPLACE_EXISTING) TestUtils.incrementalAlterConfigs(servers, adminClients.head, oldTruststoreProps, perBrokerConfig = true).all.get() - verifySslProduceConsume(sslProperties1, "alter-truststore-4") - verifySslProduceConsume(sslProperties2, "alter-truststore-5") + TestUtils.retry(30000) { + try { + verifySslProduceConsume(sslProperties1, next_group_name()) + verifySslProduceConsume(sslProperties2, next_group_name()) + } catch { + case t: Throwable => throw new AssertionError(t) + } + } // Update internal keystore/truststore and validate new client connections from broker (e.g. controller). // Alter internal keystore from `sslProperties1` to `sslProperties2`, force disconnect of a controller connection @@ -475,25 +509,28 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup val props2 = securityProps(sslProperties2, KEYSTORE_PROPS, prefix) props2 ++= securityProps(combinedStoreProps, TRUSTSTORE_PROPS, prefix) TestUtils.incrementalAlterConfigs(servers, adminClients.head, props2, perBrokerConfig = true).all.get(15, TimeUnit.SECONDS) - verifySslProduceConsume(sslProperties2, "alter-truststore-6") + verifySslProduceConsume(sslProperties2, next_group_name()) props2 ++= securityProps(sslProperties2, TRUSTSTORE_PROPS, prefix) TestUtils.incrementalAlterConfigs(servers, adminClients.head, props2, perBrokerConfig = true).all.get(15, TimeUnit.SECONDS) - verifySslProduceConsume(sslProperties2, "alter-truststore-7") + verifySslProduceConsume(sslProperties2, next_group_name()) waitForAuthenticationFailure(producerBuilder.keyStoreProps(sslProperties1)) - val controller = servers.find(_.config.brokerId == TestUtils.waitUntilControllerElected(zkClient)).get - val controllerChannelManager = controller.kafkaController.controllerChannelManager - val brokerStateInfo: mutable.HashMap[Int, ControllerBrokerStateInfo] = - JTestUtils.fieldValue(controllerChannelManager, classOf[ControllerChannelManager], "brokerStateInfo") - brokerStateInfo(0).networkClient.disconnect("0") - TestUtils.createTopic(zkClient, "testtopic2", numPartitions, replicationFactor = numServers, servers) + if (!isKRaftTest()) { + val controller = servers.find(_.config.brokerId == TestUtils.waitUntilControllerElected(zkClient)).get.asInstanceOf[KafkaServer] + val controllerChannelManager = controller.kafkaController.controllerChannelManager + val brokerStateInfo: mutable.HashMap[Int, ControllerBrokerStateInfo] = + JTestUtils.fieldValue(controllerChannelManager, classOf[ControllerChannelManager], "brokerStateInfo") + brokerStateInfo(0).networkClient.disconnect("0") + TestUtils.createTopic(zkClient, "testtopic2", numPartitions, replicationFactor = numServers, servers) - // validate that the brokerToController request works fine - verifyBrokerToControllerCall(controller) + // validate that the brokerToController request works fine + verifyBrokerToControllerCall(controller) + } } - @Test - def testLogCleanerConfig(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testLogCleanerConfig(quorum: String): Unit = { val (producerThread, consumerThread) = startProduceConsume(retries = 0) verifyThreads("kafka-log-cleaner-thread-", countPerBroker = 1) @@ -537,13 +574,23 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup stopAndVerifyProduceConsume(producerThread, consumerThread) } - @Test - def testConsecutiveConfigChange(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testConsecutiveConfigChange(quorum: String): Unit = { val topic2 = "testtopic2" val topicProps = new Properties topicProps.put(KafkaConfig.MinInSyncReplicasProp, "2") - TestUtils.createTopic(zkClient, topic2, 1, replicationFactor = numServers, servers, topicProps) - var log = servers.head.logManager.getLog(new TopicPartition(topic2, 0)).getOrElse(throw new IllegalStateException("Log not found")) + TestUtils.createTopicWithAdmin(adminClients.head, topic2, servers, numPartitions = 1, replicationFactor = numServers, topicConfig = topicProps) + + def getLogOrThrow(tp: TopicPartition): UnifiedLog = { + var (logOpt, found) = TestUtils.computeUntilTrue { + servers.head.logManager.getLog(tp) + }(_.isDefined) + assertTrue(found, "Log not found") + logOpt.get + } + + var log = getLogOrThrow(new TopicPartition(topic2, 0)) assertTrue(log.config.overriddenConfigs.contains(KafkaConfig.MinInSyncReplicasProp)) assertEquals("2", log.config.originals().get(KafkaConfig.MinInSyncReplicasProp).toString) @@ -558,7 +605,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } } - log = servers.head.logManager.getLog(new TopicPartition(topic2, 0)).getOrElse(throw new IllegalStateException("Log not found")) + log = getLogOrThrow(new TopicPartition(topic2, 0)) assertTrue(log.config.overriddenConfigs.contains(KafkaConfig.MinInSyncReplicasProp)) assertEquals("2", log.config.originals().get(KafkaConfig.MinInSyncReplicasProp).toString) // Verify topic-level config survives @@ -566,7 +613,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props.clear() props.put(KafkaConfig.LogRetentionTimeMillisProp, "604800000") reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogRetentionTimeMillisProp, "604800000")) - log = servers.head.logManager.getLog(new TopicPartition(topic2, 0)).getOrElse(throw new IllegalStateException("Log not found")) + log = getLogOrThrow(new TopicPartition(topic2, 0)) assertTrue(log.config.overriddenConfigs.contains(KafkaConfig.MinInSyncReplicasProp)) assertEquals("2", log.config.originals().get(KafkaConfig.MinInSyncReplicasProp).toString) // Verify topic-level config still survives } @@ -974,6 +1021,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } @Test + // Modifying advertised listeners is not supported in KRaft def testAdvertisedListenerUpdate(): Unit = { val adminClient = adminClients.head val externalAdminClient = createAdminClient(SecurityProtocol.SASL_SSL, SecureExternal) @@ -994,11 +1042,13 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } // Verify that endpoints have been updated in ZK for all brokers - servers.foreach(validateEndpointsInZooKeeper(_, endpoints => endpoints.contains(invalidHost))) + servers.foreach { server => + validateEndpointsInZooKeeper(server.asInstanceOf[KafkaServer], endpoints => endpoints.contains(invalidHost)) + } // Trigger session expiry and ensure that controller registers new advertised listener after expiry val controllerEpoch = zkClient.getControllerEpoch - val controllerServer = servers(zkClient.getControllerId.getOrElse(throw new IllegalStateException("No controller"))) + val controllerServer = servers(zkClient.getControllerId.getOrElse(throw new IllegalStateException("No controller"))).asInstanceOf[KafkaServer] val controllerZkClient = controllerServer.zkClient val sessionExpiringClient = createZooKeeperClientToTriggerSessionExpiry(controllerZkClient.currentZooKeeper) sessionExpiringClient.close() @@ -1022,7 +1072,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup .getCause.isInstanceOf[org.apache.kafka.common.errors.TimeoutException]) alterAdvertisedListener(adminClient, externalAdminClient, invalidHost, "localhost") - servers.foreach(validateEndpointsInZooKeeper(_, endpoints => !endpoints.contains(invalidHost))) + servers.foreach { server => + validateEndpointsInZooKeeper(server.asInstanceOf[KafkaServer], endpoints => !endpoints.contains(invalidHost)) + } // Verify that produce/consume work now val topic2 = "testtopic2" @@ -1119,7 +1171,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup assertTrue(partitions.exists(_.leader == null), "Did not find partitions with no leader") } - private def addListener(servers: Seq[KafkaServer], listenerName: String, securityProtocol: SecurityProtocol, + private def addListener(servers: Seq[KafkaBroker], listenerName: String, securityProtocol: SecurityProtocol, saslMechanisms: Seq[String]): Unit = { val config = servers.head.config val existingListenerCount = config.listeners.size @@ -1264,11 +1316,11 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup verifyProduceConsume(producer, consumer, numRecords = 10, topic) } - private def hasListenerMetric(server: KafkaServer, listenerName: String): Boolean = { + private def hasListenerMetric(server: KafkaBroker, listenerName: String): Boolean = { server.socketServer.metrics.metrics.keySet.asScala.exists(_.tags.get("listener") == listenerName) } - private def fetchBrokerConfigsFromZooKeeper(server: KafkaServer): Properties = { + private def fetchBrokerConfigsFromZooKeeper(server: KafkaBroker): Properties = { val props = adminZkClient.fetchEntityConfig(ConfigType.Broker, server.config.brokerId.toString) server.config.dynamicConfig.fromPersistentProps(props, perBrokerConfig = true) } @@ -1322,7 +1374,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup }, "Did not fail authentication with invalid config") } - private def describeConfig(adminClient: Admin, servers: Seq[KafkaServer] = this.servers): Config = { + private def describeConfig(adminClient: Admin, servers: Seq[KafkaBroker] = this.servers): Config = { val configResources = servers.map { server => new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString) } @@ -1419,7 +1471,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } @nowarn("cat=deprecation") - private def alterConfigsOnServer(server: KafkaServer, props: Properties): Unit = { + private def alterConfigsOnServer(server: KafkaBroker, props: Properties): Unit = { val configEntries = props.asScala.map { case (k, v) => new ConfigEntry(k, v) }.toList.asJava val newConfig = new Config(configEntries) val configs = Map(new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString) -> newConfig).asJava @@ -1428,7 +1480,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } @nowarn("cat=deprecation") - private def alterConfigs(servers: Seq[KafkaServer], adminClient: Admin, props: Properties, + private def alterConfigs(servers: Seq[KafkaBroker], adminClient: Admin, props: Properties, perBrokerConfig: Boolean): AlterConfigsResult = { val configEntries = props.asScala.map { case (k, v) => new ConfigEntry(k, v) }.toList.asJava val newConfig = new Config(configEntries) @@ -1507,7 +1559,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup private def createPasswordEncoder(config: KafkaConfig, secret: Option[Password]): PasswordEncoder = { val encoderSecret = secret.getOrElse(throw new IllegalStateException("Password encoder secret not configured")) - new PasswordEncoder(encoderSecret, + PasswordEncoder.encrypting(encoderSecret, config.passwordEncoderKeyFactoryAlgorithm, config.passwordEncoderCipherAlgorithm, config.passwordEncoderKeyLength, @@ -1518,7 +1570,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup servers.foreach { server => waitForConfigOnServer(server, propName, propValue, maxWaitMs) } } - private def waitForConfigOnServer(server: KafkaServer, propName: String, propValue: String, maxWaitMs: Long = 10000): Unit = { + private def waitForConfigOnServer(server: KafkaBroker, propName: String, propValue: String, maxWaitMs: Long = 10000): Unit = { TestUtils.retry(maxWaitMs) { assertEquals(propValue, server.config.originals.get(propName)) } diff --git a/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala b/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala new file mode 100644 index 0000000000000..822099f605b01 --- /dev/null +++ b/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala @@ -0,0 +1,194 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package integration.kafka.server + +import kafka.server.{BaseFetchRequestTest, KafkaConfig} +import kafka.utils.{TestInfoUtils, TestUtils} +import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.requests.FetchResponse +import org.apache.kafka.common.serialization.ByteArrayDeserializer +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.{Test, Timeout} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource + +import java.util.Properties +import scala.jdk.CollectionConverters._ + +class FetchFromFollowerIntegrationTest extends BaseFetchRequestTest { + val numNodes = 2 + val numParts = 1 + + val topic = "test-fetch-from-follower" + val leaderBrokerId = 0 + val followerBrokerId = 1 + + def overridingProps: Properties = { + val props = new Properties + props.put(KafkaConfig.NumPartitionsProp, numParts.toString) + props.put(KafkaConfig.OffsetsTopicReplicationFactorProp, numNodes.toString) + props + } + + override def generateConfigs: collection.Seq[KafkaConfig] = { + TestUtils.createBrokerConfigs(numNodes, zkConnectOrNull, enableControlledShutdown = false, enableFetchFromFollower = true) + .map(KafkaConfig.fromProps(_, overridingProps)) + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + @Timeout(15) + def testFollowerCompleteDelayedFetchesOnReplication(quorum: String): Unit = { + // Create a topic with 2 replicas where broker 0 is the leader and 1 is the follower. + val admin = createAdminClient() + TestUtils.createTopicWithAdmin( + admin, + topic, + brokers, + replicaAssignment = Map(0 -> Seq(leaderBrokerId, followerBrokerId)) + ) + + val version = ApiKeys.FETCH.latestVersion() + val topicPartition = new TopicPartition(topic, 0) + val offsetMap = Map(topicPartition -> 0L) + + // Set fetch.max.wait.ms to a value (20 seconds) greater than the timeout (15 seconds). + // Send a fetch request before the record is replicated to ensure that the replication + // triggers purgatory completion. + val fetchRequest = createConsumerFetchRequest( + maxResponseBytes = 1000, + maxPartitionBytes = 1000, + Seq(topicPartition), + offsetMap, + version, + maxWaitMs = 20000, + minBytes = 1 + ) + + val socket = connect(brokerSocketServer(followerBrokerId)) + try { + send(fetchRequest, socket) + TestUtils.generateAndProduceMessages(brokers, topic, numMessages = 1) + val response = receive[FetchResponse](socket, ApiKeys.FETCH, version) + assertEquals(Errors.NONE, response.error) + assertEquals(Map(Errors.NONE -> 2).asJava, response.errorCounts) + } finally { + socket.close() + } + } + + @Test + def testFetchFromLeaderWhilePreferredReadReplicaIsUnavailable(): Unit = { + // Create a topic with 2 replicas where broker 0 is the leader and 1 is the follower. + val admin = createAdminClient() + TestUtils.createTopicWithAdmin( + admin, + topic, + brokers, + replicaAssignment = Map(0 -> Seq(leaderBrokerId, followerBrokerId)) + ) + + TestUtils.generateAndProduceMessages(brokers, topic, numMessages = 10) + + assertEquals(1, getPreferredReplica) + + // Shutdown follower broker. + brokers(followerBrokerId).shutdown() + val topicPartition = new TopicPartition(topic, 0) + TestUtils.waitUntilTrue(() => { + val endpoints = brokers(leaderBrokerId).metadataCache.getPartitionReplicaEndpoints(topicPartition, listenerName) + !endpoints.contains(followerBrokerId) + }, "follower is still reachable.") + + assertEquals(-1, getPreferredReplica) + } + + @Test + def testFetchFromFollowerWithRoll(): Unit = { + // Create a topic with 2 replicas where broker 0 is the leader and 1 is the follower. + val admin = createAdminClient() + TestUtils.createTopicWithAdmin( + admin, + topic, + brokers, + replicaAssignment = Map(0 -> Seq(leaderBrokerId, followerBrokerId)) + ) + + // Create consumer with client.rack = follower id. + val consumerProps = new Properties + consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()) + consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "test-group") + consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + consumerProps.put(ConsumerConfig.CLIENT_RACK_CONFIG, followerBrokerId.toString) + val consumer = new KafkaConsumer(consumerProps, new ByteArrayDeserializer, new ByteArrayDeserializer) + try { + consumer.subscribe(List(topic).asJava) + + // Wait until preferred replica is set to follower. + TestUtils.waitUntilTrue(() => { + getPreferredReplica == 1 + }, "Preferred replica is not set") + + // Produce and consume. + TestUtils.generateAndProduceMessages(brokers, topic, numMessages = 1) + TestUtils.pollUntilAtLeastNumRecords(consumer, 1) + + // Shutdown follower, produce and consume should work. + brokers(followerBrokerId).shutdown() + TestUtils.generateAndProduceMessages(brokers, topic, numMessages = 1) + TestUtils.pollUntilAtLeastNumRecords(consumer, 1) + + // Start the follower and wait until preferred replica is set to follower. + brokers(followerBrokerId).startup() + TestUtils.waitUntilTrue(() => { + getPreferredReplica == 1 + }, "Preferred replica is not set") + + // Produce and consume should still work. + TestUtils.generateAndProduceMessages(brokers, topic, numMessages = 1) + TestUtils.pollUntilAtLeastNumRecords(consumer, 1) + } finally { + consumer.close() + } + } + + private def getPreferredReplica: Int = { + val topicPartition = new TopicPartition(topic, 0) + val offsetMap = Map(topicPartition -> 0L) + + val request = createConsumerFetchRequest( + maxResponseBytes = 1000, + maxPartitionBytes = 1000, + Seq(topicPartition), + offsetMap, + ApiKeys.FETCH.latestVersion, + maxWaitMs = 500, + minBytes = 1, + rackId = followerBrokerId.toString + ) + val response = connectAndReceive[FetchResponse](request, brokers(leaderBrokerId).socketServer) + assertEquals(Errors.NONE, response.error) + assertEquals(Map(Errors.NONE -> 2).asJava, response.errorCounts) + assertEquals(1, response.data.responses.size) + val topicResponse = response.data.responses.get(0) + assertEquals(1, topicResponse.partitions.size) + + topicResponse.partitions.get(0).preferredReadReplica + } +} diff --git a/core/src/test/scala/integration/kafka/server/FetchRequestBetweenDifferentIbpTest.scala b/core/src/test/scala/integration/kafka/server/FetchRequestBetweenDifferentIbpTest.scala index 36d9c00bfd207..f4fd79b50936a 100644 --- a/core/src/test/scala/integration/kafka/server/FetchRequestBetweenDifferentIbpTest.scala +++ b/core/src/test/scala/integration/kafka/server/FetchRequestBetweenDifferentIbpTest.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package integration.kafka.server +package kafka.server import java.time.Duration import java.util.Arrays.asList -import kafka.server.{BaseRequestTest, KafkaConfig} import kafka.utils.TestUtils import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.common.TopicPartition diff --git a/core/src/test/scala/integration/kafka/server/FetchRequestTestDowngrade.scala b/core/src/test/scala/integration/kafka/server/FetchRequestTestDowngrade.scala index c714b8cc3689a..25867cda7e5c9 100644 --- a/core/src/test/scala/integration/kafka/server/FetchRequestTestDowngrade.scala +++ b/core/src/test/scala/integration/kafka/server/FetchRequestTestDowngrade.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package integration.kafka.server +package kafka.server import java.time.Duration import java.util.Arrays.asList -import kafka.server.{BaseRequestTest, KafkaConfig} import kafka.utils.TestUtils import kafka.zk.ZkVersion import org.apache.kafka.clients.producer.ProducerRecord diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 509facf921189..45ab38ef4f6ad 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -21,7 +21,7 @@ import kafka.network.SocketServer import kafka.server.IntegrationTestUtils.connectAndReceive import kafka.testkit.{BrokerNode, KafkaClusterTestKit, TestKitNodes} import kafka.utils.TestUtils -import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, Config, ConfigEntry, NewPartitionReassignment, NewTopic} +import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp, Config, ConfigEntry, DescribeMetadataQuorumOptions, FeatureUpdate, NewPartitionReassignment, NewTopic, UpdateFeaturesOptions} import org.apache.kafka.common.{TopicPartition, TopicPartitionInfo} import org.apache.kafka.common.message.DescribeClusterRequestData import org.apache.kafka.common.network.ListenerName @@ -32,7 +32,7 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{Tag, Test, Timeout} import java.util -import java.util.{Arrays, Collections, Optional} +import java.util.{Arrays, Collections, Optional, OptionalLong, Properties} import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource.Type @@ -296,13 +296,14 @@ class KRaftClusterTest { @Test def testCreateClusterInvalidMetadataVersion(): Unit = { - assertThrows(classOf[IllegalArgumentException], () => { - new KafkaClusterTestKit.Builder( - new TestKitNodes.Builder(). - setBootstrapMetadataVersion(MetadataVersion.IBP_2_7_IV0). - setNumBrokerNodes(1). - setNumControllerNodes(1).build()).build() - }) + assertEquals("Bootstrap metadata versions before 3.3-IV0 are not supported. Can't load " + + "metadata from testkit", assertThrows(classOf[RuntimeException], () => { + new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setBootstrapMetadataVersion(MetadataVersion.IBP_2_7_IV0). + setNumBrokerNodes(1). + setNumControllerNodes(1).build()).build() + }).getMessage) } private def doOnStartedKafkaCluster(numControllerNodes: Int = 1, @@ -426,15 +427,17 @@ class KRaftClusterTest { }, "Timed out waiting for replica assignments for topic foo. " + s"Wanted: ${expectedMapping}. Got: ${currentMapping}") - checkReplicaManager( - cluster, - List( - (0, List(true, true, false, true)), - (1, List(true, true, false, true)), - (2, List(true, true, true, true)), - (3, List(false, false, true, true)) + TestUtils.retry(60000) { + checkReplicaManager( + cluster, + List( + (0, List(true, true, false, true)), + (1, List(true, true, false, true)), + (2, List(true, true, true, true)), + (3, List(false, false, true, true)) + ) ) - ) + } } finally { admin.close() } @@ -778,4 +781,96 @@ class KRaftClusterTest { cluster.close() } } + + def createAdminClient(cluster: KafkaClusterTestKit): Admin = { + var props: Properties = null + props = cluster.clientProperties() + props.put(AdminClientConfig.CLIENT_ID_CONFIG, this.getClass.getName) + Admin.create(props) + } + + @Test + def testDescribeQuorumRequestToBrokers() : Unit = { + val cluster = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(4). + setNumControllerNodes(3).build()).build() + try { + cluster.format + cluster.startup + for (i <- 0 to 3) { + TestUtils.waitUntilTrue(() => cluster.brokers.get(i).brokerState == BrokerState.RUNNING, + "Broker Never started up") + } + val admin = createAdminClient(cluster) + try { + val quorumState = admin.describeMetadataQuorum(new DescribeMetadataQuorumOptions) + val quorumInfo = quorumState.quorumInfo.get() + + assertEquals(cluster.controllers.asScala.keySet, quorumInfo.voters.asScala.map(_.replicaId).toSet) + assertTrue(cluster.controllers.asScala.keySet.contains(quorumInfo.leaderId), + s"Leader ID ${quorumInfo.leaderId} was not a controller ID.") + + val (voters, voterResponseValid) = + TestUtils.computeUntilTrue( + admin.describeMetadataQuorum(new DescribeMetadataQuorumOptions) + .quorumInfo().get().voters() + ) { voters => voters.stream + .allMatch(voter => (voter.logEndOffset > 0 + && voter.lastFetchTimestamp() != OptionalLong.empty() + && voter.lastCaughtUpTimestamp() != OptionalLong.empty())) + } + + assertTrue(voterResponseValid, s"At least one voter did not return the expected state within timeout." + + s"The responses gathered for all the voters: ${voters.toString}") + + val (observers, observerResponseValid) = + TestUtils.computeUntilTrue( + admin.describeMetadataQuorum(new DescribeMetadataQuorumOptions) + .quorumInfo().get().observers() + ) { observers => + ( + cluster.brokers.asScala.keySet == observers.asScala.map(_.replicaId).toSet + && observers.stream.allMatch(observer => (observer.logEndOffset > 0 + && observer.lastFetchTimestamp() != OptionalLong.empty() + && observer.lastCaughtUpTimestamp() != OptionalLong.empty()))) + } + + assertTrue(observerResponseValid, s"At least one observer did not return the expected state within timeout." + + s"The responses gathered for all the observers: ${observers.toString}") + } finally { + admin.close() + } + } finally { + cluster.close() + } + } + + + @Test + def testUpdateMetadataVersion(): Unit = { + val cluster = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setBootstrapMetadataVersion(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION). + setNumBrokerNodes(4). + setNumControllerNodes(3).build()).build() + try { + cluster.format() + cluster.startup() + cluster.waitForReadyBrokers() + val admin = Admin.create(cluster.clientProperties()) + try { + admin.updateFeatures( + Map(MetadataVersion.FEATURE_NAME -> + new FeatureUpdate(MetadataVersion.latest().featureLevel(), FeatureUpdate.UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions + ) + } finally { + admin.close() + } + TestUtils.waitUntilTrue(() => cluster.brokers().get(1).metadataCache.currentImage().features().metadataVersion().equals(MetadataVersion.latest()), + "Timed out waiting for metadata version update.") + } finally { + cluster.close() + } + } } diff --git a/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala b/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala index c060e3a6daae4..81810c61dac22 100644 --- a/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package integration.kafka.server +package kafka.server import kafka.test.ClusterInstance import kafka.test.annotation.{ClusterTest, ClusterTests, Type} @@ -32,9 +32,9 @@ import scala.jdk.CollectionConverters._ @ExtendWith(value = Array(classOf[ClusterTestExtensions])) class MetadataVersionIntegrationTest { @ClusterTests(value = Array( - new ClusterTest(clusterType = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_0_IV1), - new ClusterTest(clusterType = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_1_IV0), - new ClusterTest(clusterType = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_2_IV0) + new ClusterTest(clusterType = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_3_IV0), + new ClusterTest(clusterType = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_3_IV1), + new ClusterTest(clusterType = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_3_IV2) )) def testBasicMetadataVersionUpgrade(clusterInstance: ClusterInstance): Unit = { val admin = clusterInstance.createAdminClient() @@ -44,7 +44,7 @@ class MetadataVersionIntegrationTest { assertEquals(ff.maxVersionLevel(), clusterInstance.config().metadataVersion().featureLevel()) // Update to new version - val updateVersion = MetadataVersion.IBP_3_3_IV0.featureLevel.shortValue + val updateVersion = MetadataVersion.IBP_3_3_IV3.featureLevel.shortValue val updateResult = admin.updateFeatures( Map("metadata.version" -> new FeatureUpdate(updateVersion, UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions()) updateResult.all().get() diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index b82f86a8cb335..f456770f0f746 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -24,6 +24,7 @@ import java.util.{Collections, Properties} import java.util.concurrent.CompletableFuture import javax.security.auth.login.Configuration import kafka.raft.KafkaRaftManager +import kafka.server.metadata.BrokerServerMetrics import kafka.tools.StorageTool import kafka.utils.{CoreUtils, Logging, TestInfoUtils, TestUtils} import kafka.zk.{AdminZkClient, EmbeddedZookeeper, KafkaZkClient} @@ -32,10 +33,13 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{Exit, Time} -import org.apache.kafka.controller.BootstrapMetadata +import org.apache.kafka.controller.QuorumControllerMetrics import org.apache.kafka.metadata.MetadataRecordSerde +import org.apache.kafka.metadata.bootstrap.BootstrapMetadata import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec} import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} +import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler} +import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.{WatchedEvent, Watcher, ZooKeeper} import org.junit.jupiter.api.Assertions._ @@ -44,12 +48,14 @@ import org.junit.jupiter.api.{AfterAll, AfterEach, BeforeAll, BeforeEach, Tag, T import scala.collection.mutable.ListBuffer import scala.collection.{Seq, immutable} import scala.compat.java8.OptionConverters._ -import scala.jdk.CollectionConverters._ trait QuorumImplementation { - def createBroker(config: KafkaConfig, - time: Time, - startup: Boolean): KafkaBroker + def createBroker( + config: KafkaConfig, + time: Time = Time.SYSTEM, + startup: Boolean = true, + threadNamePrefix: Option[String] = None, + ): KafkaBroker def shutdown(): Unit } @@ -61,10 +67,13 @@ class ZooKeeperQuorumImplementation( val adminZkClient: AdminZkClient, val log: Logging ) extends QuorumImplementation { - override def createBroker(config: KafkaConfig, - time: Time, - startup: Boolean): KafkaBroker = { - val server = new KafkaServer(config, time, None, false) + override def createBroker( + config: KafkaConfig, + time: Time, + startup: Boolean, + threadNamePrefix: Option[String], + ): KafkaBroker = { + val server = new KafkaServer(config, time, threadNamePrefix, false) if (startup) server.startup() server } @@ -75,23 +84,34 @@ class ZooKeeperQuorumImplementation( } } -class KRaftQuorumImplementation(val raftManager: KafkaRaftManager[ApiMessageAndVersion], - val controllerServer: ControllerServer, - val metadataDir: File, - val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], - val clusterId: String, - val log: Logging) extends QuorumImplementation { - override def createBroker(config: KafkaConfig, - time: Time, - startup: Boolean): KafkaBroker = { +class KRaftQuorumImplementation( + val raftManager: KafkaRaftManager[ApiMessageAndVersion], + val controllerServer: ControllerServer, + val metadataDir: File, + val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], + val clusterId: String, + val log: Logging, + val faultHandler: FaultHandler +) extends QuorumImplementation { + override def createBroker( + config: KafkaConfig, + time: Time, + startup: Boolean, + threadNamePrefix: Option[String], + ): KafkaBroker = { + val metrics = new Metrics() val broker = new BrokerServer(config = config, metaProps = new MetaProperties(clusterId, config.nodeId), raftManager = raftManager, time = time, - metrics = new Metrics(), + metrics = metrics, + brokerMetrics = BrokerServerMetrics(metrics), threadNamePrefix = Some("Broker%02d_".format(config.nodeId)), initialOfflineDirs = Seq(), - controllerQuorumVotersFuture = controllerQuorumVotersFuture) + controllerQuorumVotersFuture = controllerQuorumVotersFuture, + fatalFaultHandler = faultHandler, + metadataLoadingFaultHandler = faultHandler, + metadataPublishingFaultHandler = faultHandler) if (startup) broker.startup() broker } @@ -122,10 +142,10 @@ abstract class QuorumTestHarness extends Logging { protected def metadataVersion: MetadataVersion = MetadataVersion.latest() - val bootstrapRecords: ListBuffer[ApiMessageAndVersion] = ListBuffer() + private var testInfo: TestInfo = _ + private var implementation: QuorumImplementation = _ - private var testInfo: TestInfo = null - private var implementation: QuorumImplementation = null + val bootstrapRecords: ListBuffer[ApiMessageAndVersion] = ListBuffer() def isKRaftTest(): Boolean = { TestInfoUtils.isKRaft(testInfo) @@ -179,6 +199,8 @@ abstract class QuorumTestHarness extends Logging { } } + val faultHandler = new MockFaultHandler("quorumTestHarnessFaultHandler") + // Note: according to the junit documentation: "JUnit Jupiter does not guarantee the execution // order of multiple @BeforeEach methods that are declared within a single test class or test // interface." Therefore, if you have things you would like to do before each test case runs, it @@ -219,10 +241,13 @@ abstract class QuorumTestHarness extends Logging { } } - def createBroker(config: KafkaConfig, - time: Time = Time.SYSTEM, - startup: Boolean = true): KafkaBroker = { - implementation.createBroker(config, time, startup) + def createBroker( + config: KafkaConfig, + time: Time = Time.SYSTEM, + startup: Boolean = true, + threadNamePrefix: Option[String] = None + ): KafkaBroker = { + implementation.createBroker(config, time, startup, threadNamePrefix) } def shutdownZooKeeper(): Unit = asZk().shutdown() @@ -291,11 +316,14 @@ abstract class QuorumTestHarness extends Logging { raftManager = raftManager, time = Time.SYSTEM, metrics = controllerMetrics, + controllerMetrics = new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry(), Time.SYSTEM), threadNamePrefix = Option(threadNamePrefix), controllerQuorumVotersFuture = controllerQuorumVotersFuture, configSchema = KafkaRaftServer.configSchema, raftApiVersions = raftManager.apiVersions, - bootstrapMetadata = BootstrapMetadata.create(metadataVersion, bootstrapRecords.asJava), + bootstrapMetadata = BootstrapMetadata.fromVersion(metadataVersion, "test harness"), + metadataFaultHandler = faultHandler, + fatalFaultHandler = faultHandler ) controllerServer.socketServerFirstBoundPortFuture.whenComplete((port, e) => { if (e != null) { @@ -319,7 +347,8 @@ abstract class QuorumTestHarness extends Logging { metadataDir, controllerQuorumVotersFuture, metaProperties.clusterId, - this) + this, + faultHandler) } private def newZooKeeperQuorum(): ZooKeeperQuorumImplementation = { @@ -362,6 +391,7 @@ abstract class QuorumTestHarness extends Logging { } System.clearProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) Configuration.setConfiguration(null) + faultHandler.maybeRethrowFirstException() } // Trigger session expiry by reusing the session id in another client diff --git a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala index 503ce7d2beefd..f8dccd17d0d8a 100644 --- a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala +++ b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala @@ -78,7 +78,8 @@ class RaftClusterSnapshotTest { raftManager.replicatedLog.latestSnapshot.get(), new MetadataRecordSerde(), BufferSupplier.create(), - 1 + 1, + true ) ) { snapshot => // Check that the snapshot is non-empty diff --git a/core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala b/core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala index 4f673cdd60ada..c64d25fe4e6a6 100644 --- a/core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/tools/MirrorMakerIntegrationTest.scala @@ -18,26 +18,27 @@ package kafka.tools import java.util.Properties import java.util.concurrent.atomic.AtomicBoolean - import scala.collection.Seq import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig import kafka.tools.MirrorMaker.{ConsumerWrapper, MirrorMakerProducer, NoRecordsException} -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.TimeoutException import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer} import org.apache.kafka.common.utils.Exit -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource @deprecated(message = "Use the Connect-based MirrorMaker instead (aka MM2).", since = "3.0") class MirrorMakerIntegrationTest extends KafkaServerTestHarness { override def generateConfigs: Seq[KafkaConfig] = - TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps(_, new Properties())) + TestUtils.createBrokerConfigs(1, zkConnectOrNull).map(KafkaConfig.fromProps(_, new Properties())) val exited = new AtomicBoolean(false) @@ -57,8 +58,9 @@ class MirrorMakerIntegrationTest extends KafkaServerTestHarness { } } - @Test - def testCommitOffsetsThrowTimeoutException(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCommitOffsetsThrowTimeoutException(quorum: String): Unit = { val consumerProps = new Properties consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "test-group") consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") @@ -70,8 +72,9 @@ class MirrorMakerIntegrationTest extends KafkaServerTestHarness { assertThrows(classOf[TimeoutException], () => mirrorMakerConsumer.commit()) } - @Test - def testCommitOffsetsRemoveNonExistentTopics(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCommitOffsetsRemoveNonExistentTopics(quorum: String): Unit = { val consumerProps = new Properties consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "test-group") consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") @@ -85,8 +88,9 @@ class MirrorMakerIntegrationTest extends KafkaServerTestHarness { assertTrue(mirrorMakerConsumer.offsets.isEmpty, "Offsets for non-existent topics should be removed") } - @Test - def testCommaSeparatedRegex(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCommaSeparatedRegex(quorum: String): Unit = { val topic = "new-topic" val msg = "a test message" diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 94bf45391ed3f..fa9885e35837d 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -16,7 +16,7 @@ */ package kafka.raft -import kafka.log.{Defaults, UnifiedLog, SegmentDeletion} +import kafka.log.{Defaults, SegmentDeletion, UnifiedLog} import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, MetadataLogSegmentMillisProp, MetadataLogSegmentMinBytesProp, NodeIdProp, ProcessRolesProp, QuorumVotersProp} import kafka.server.{KafkaConfig, KafkaRaftServer} import kafka.utils.{MockTime, TestUtils} @@ -28,7 +28,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.raft.internals.BatchBuilder import org.apache.kafka.raft._ import org.apache.kafka.server.common.serialization.RecordSerde -import org.apache.kafka.snapshot.{RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} +import org.apache.kafka.snapshot.{FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} import org.apache.kafka.test.TestUtils.assertOptional import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -66,13 +66,13 @@ final class KafkaMetadataLogTest { props.put(MetadataLogSegmentMillisProp, Int.box(10 * 1024)) assertThrows(classOf[InvalidConfigurationException], () => { val kafkaConfig = KafkaConfig.fromProps(props) - val metadataConfig = MetadataLogConfig.apply(kafkaConfig, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES) + val metadataConfig = MetadataLogConfig(kafkaConfig, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES) buildMetadataLog(tempDir, mockTime, metadataConfig) }) props.put(MetadataLogSegmentMinBytesProp, Int.box(10240)) val kafkaConfig = KafkaConfig.fromProps(props) - val metadataConfig = MetadataLogConfig.apply(kafkaConfig, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES) + val metadataConfig = MetadataLogConfig(kafkaConfig, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES) buildMetadataLog(tempDir, mockTime, metadataConfig) } @@ -391,6 +391,87 @@ final class KafkaMetadataLogTest { } } + @Test + def testStartupWithInvalidSnapshotState(): Unit = { + // Initialize an empty log at offset 100. + var log = buildMetadataLog(tempDir, mockTime) + log.log.truncateFullyAndStartAt(newOffset = 100) + log.close() + + val metadataDir = metadataLogDir(tempDir) + assertTrue(metadataDir.exists()) + + // Initialization should fail unless we have a snapshot at an offset + // greater than or equal to 100. + assertThrows(classOf[IllegalStateException], () => { + buildMetadataLog(tempDir, mockTime) + }) + // Snapshots at offsets less than 100 are not sufficient. + writeEmptySnapshot(metadataDir, new OffsetAndEpoch(50, 1)) + assertThrows(classOf[IllegalStateException], () => { + buildMetadataLog(tempDir, mockTime) + }) + + // Snapshot at offset 100 should be fine. + writeEmptySnapshot(metadataDir, new OffsetAndEpoch(100, 1)) + log = buildMetadataLog(tempDir, mockTime) + log.log.truncateFullyAndStartAt(newOffset = 200) + log.close() + + // Snapshots at higher offsets are also fine. In this case, the + // log start offset should advance to the first snapshot offset. + writeEmptySnapshot(metadataDir, new OffsetAndEpoch(500, 1)) + log = buildMetadataLog(tempDir, mockTime) + assertEquals(500, log.log.logStartOffset) + } + + @Test + def testSnapshotDeletionWithInvalidSnapshotState(): Unit = { + // Initialize an empty log at offset 100. + val log = buildMetadataLog(tempDir, mockTime) + log.log.truncateFullyAndStartAt(newOffset = 100) + log.close() + + val metadataDir = metadataLogDir(tempDir) + assertTrue(metadataDir.exists()) + + // We have one deleted snapshot at an offset matching the start offset. + val snapshotId = new OffsetAndEpoch(100, 1) + writeEmptySnapshot(metadataDir, snapshotId) + + val deletedPath = Snapshots.markForDelete(metadataDir.toPath, snapshotId) + assertTrue(deletedPath.toFile.exists()) + + // Initialization should still fail. + assertThrows(classOf[IllegalStateException], () => { + buildMetadataLog(tempDir, mockTime) + }) + + // The snapshot marked for deletion should still exist. + assertTrue(deletedPath.toFile.exists()) + } + + private def metadataLogDir( + logDir: File + ): File = { + new File( + logDir.getAbsolutePath, + UnifiedLog.logDirName(KafkaRaftServer.MetadataPartition) + ) + } + + private def writeEmptySnapshot( + metadataDir: File, + snapshotId: OffsetAndEpoch + ): Unit = { + val writer = FileRawSnapshotWriter.create( + metadataDir.toPath, + snapshotId, + Optional.empty() + ) + TestUtils.resource(writer)(_.freeze()) + } + @Test def testDoesntTruncateFully(): Unit = { val log = buildMetadataLog(tempDir, mockTime) @@ -869,6 +950,56 @@ final class KafkaMetadataLogTest { }) }) } + + @Test + def testSegmentsLessThanLatestSnapshot(): Unit = { + val config = DefaultMetadataLogConfig.copy( + logSegmentBytes = 10240, + logSegmentMinBytes = 10240, + logSegmentMillis = 10 * 1000, + retentionMaxBytes = 10240, + retentionMillis = 60 * 1000, + maxBatchSizeInBytes = 200 + ) + val log = buildMetadataLog(tempDir, mockTime, config) + + // Generate enough data to cause a segment roll + for (_ <- 0 to 2000) { + append(log, 10, 1) + } + log.updateHighWatermark(new LogOffsetMetadata(log.endOffset.offset)) + + // The clean up code requires that there are at least two snapshots + // Generate first snapshots that includes the first segment by using the base offset of the second segment + val snapshotId1 = new OffsetAndEpoch( + log.log.logSegments.drop(1).head.baseOffset, + 1 + ) + TestUtils.resource(log.storeSnapshot(snapshotId1).get()) { snapshot => + snapshot.freeze() + } + // Generate second snapshots that includes the second segment by using the base offset of the third segment + val snapshotId2 = new OffsetAndEpoch( + log.log.logSegments.drop(2).head.baseOffset, + 1 + ) + TestUtils.resource(log.storeSnapshot(snapshotId2).get()) { snapshot => + snapshot.freeze() + } + + // Sleep long enough to trigger a possible segment delete because of the default retention + val defaultLogRetentionMs = Defaults.RetentionMs * 2 + mockTime.sleep(defaultLogRetentionMs) + + assertTrue(log.maybeClean()) + assertEquals(1, log.snapshotCount()) + assertTrue(log.startOffset > 0, s"${log.startOffset} must be greater than 0") + val latestSnapshotOffset = log.latestSnapshotId().get.offset + assertTrue( + latestSnapshotOffset >= log.startOffset, + s"latest snapshot offset ($latestSnapshotOffset) must be >= log start offset (${log.startOffset})" + ) + } } object KafkaMetadataLogTest { @@ -879,11 +1010,7 @@ object KafkaMetadataLogTest { override def write(data: Array[Byte], serializationCache: ObjectSerializationCache, out: Writable): Unit = { out.writeByteArray(data) } - override def read(input: protocol.Readable, size: Int): Array[Byte] = { - val array = new Array[Byte](size) - input.readArray(array) - array - } + override def read(input: protocol.Readable, size: Int): Array[Byte] = input.readArray(size) } val DefaultMetadataLogConfig = MetadataLogConfig( diff --git a/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala b/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala index df114ef59e58a..ea2b439c166ae 100644 --- a/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala +++ b/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala @@ -37,12 +37,14 @@ final class BrokerServerMetricsTest { val expectedMetrics = Set( new MetricName("last-applied-record-offset", expectedGroup, "", Collections.emptyMap()), new MetricName("last-applied-record-timestamp", expectedGroup, "", Collections.emptyMap()), - new MetricName("last-applied-record-lag-ms", expectedGroup, "", Collections.emptyMap()) + new MetricName("last-applied-record-lag-ms", expectedGroup, "", Collections.emptyMap()), + new MetricName("metadata-load-error-count", expectedGroup, "", Collections.emptyMap()), + new MetricName("metadata-apply-error-count", expectedGroup, "", Collections.emptyMap()) ) TestUtils.resource(BrokerServerMetrics(metrics)) { brokerMetrics => val metricsMap = metrics.metrics().asScala.filter{ case (name, _) => name.group == expectedGroup } - assertEquals(3, metricsMap.size) + assertEquals(expectedMetrics.size, metricsMap.size) metricsMap.foreach { case (name, metric) => assertTrue(expectedMetrics.contains(name)) } @@ -85,4 +87,36 @@ final class BrokerServerMetricsTest { assertEquals(time.milliseconds - timestamp, lagMetric.metricValue.asInstanceOf[Long]) } } + + @Test + def testMetadataLoadErrorCount(): Unit = { + val time = new MockTime() + val metrics = new Metrics(time) + TestUtils.resource(BrokerServerMetrics(metrics)) { brokerMetrics => + val metadataLoadErrorCountMetric = metrics.metrics().get(brokerMetrics.metadataLoadErrorCountName) + + assertEquals(0L, metadataLoadErrorCountMetric.metricValue.asInstanceOf[Long]) + + // Update metric value and check + val errorCount = 100 + brokerMetrics.metadataLoadErrorCount.set(errorCount) + assertEquals(errorCount, metadataLoadErrorCountMetric.metricValue.asInstanceOf[Long]) + } + } + + @Test + def testMetadataApplyErrorCount(): Unit = { + val time = new MockTime() + val metrics = new Metrics(time) + TestUtils.resource(BrokerServerMetrics(metrics)) { brokerMetrics => + val metadataApplyErrorCountMetric = metrics.metrics().get(brokerMetrics.metadataApplyErrorCountName) + + assertEquals(0L, metadataApplyErrorCountMetric.metricValue.asInstanceOf[Long]) + + // Update metric value and check + val errorCount = 100 + brokerMetrics.metadataApplyErrorCount.set(errorCount) + assertEquals(errorCount, metadataApplyErrorCountMetric.metricValue.asInstanceOf[Long]) + } + } } diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index ea4215d9c39f5..4e2bfee60ee54 100755 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -17,18 +17,24 @@ package kafka.admin -import java.util.Optional +import java.util.{Collections, Optional} import kafka.controller.ReplicaAssignment -import kafka.server.BaseRequestTest -import kafka.utils.TestUtils +import kafka.server.{BaseRequestTest, BrokerServer} +import kafka.utils.{TestInfoUtils, TestUtils} import kafka.utils.TestUtils._ -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.clients.admin.{Admin, NewPartitions, NewTopic} import org.apache.kafka.common.errors.InvalidReplicaAssignmentException import org.apache.kafka.common.requests.MetadataResponse.TopicMetadata import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} - +import org.junit.jupiter.api.{BeforeEach, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource + +import java.util +import java.util.Arrays.asList +import java.util.Collections.singletonList +import java.util.concurrent.ExecutionException import scala.jdk.CollectionConverters._ class AddPartitionsTest extends BaseRequestTest { @@ -47,44 +53,97 @@ class AddPartitionsTest extends BaseRequestTest { val topic4Assignment = Map(0 -> ReplicaAssignment(Seq(0,3), List(), List())) val topic5 = "new-topic5" val topic5Assignment = Map(1 -> ReplicaAssignment(Seq(0,1), List(), List())) + var admin: Admin = null @BeforeEach override def setUp(testInfo: TestInfo): Unit = { super.setUp(testInfo) + if (isKRaftTest()) { + brokers.foreach(broker => broker.asInstanceOf[BrokerServer].lifecycleManager.initialUnfenceFuture.get()) + } createTopicWithAssignment(topic1, partitionReplicaAssignment = topic1Assignment.map { case (k, v) => k -> v.replicas }) createTopicWithAssignment(topic2, partitionReplicaAssignment = topic2Assignment.map { case (k, v) => k -> v.replicas }) createTopicWithAssignment(topic3, partitionReplicaAssignment = topic3Assignment.map { case (k, v) => k -> v.replicas }) createTopicWithAssignment(topic4, partitionReplicaAssignment = topic4Assignment.map { case (k, v) => k -> v.replicas }) + admin = createAdminClient() } - @Test - def testWrongReplicaCount(): Unit = { - assertThrows(classOf[InvalidReplicaAssignmentException], () => adminZkClient.addPartitions(topic1, topic1Assignment, adminZkClient.getBrokerMetadatas(), 2, - Some(Map(0 -> Seq(0, 1), 1 -> Seq(0, 1, 2))))) + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testWrongReplicaCount(quorum: String): Unit = { + assertEquals(classOf[InvalidReplicaAssignmentException], assertThrows(classOf[ExecutionException], () => { + admin.createPartitions(Collections.singletonMap(topic1, + NewPartitions.increaseTo(2, singletonList(asList(0, 1, 2))))).all().get() + }).getCause.getClass) } - @Test - def testMissingPartition0(): Unit = { - val e = assertThrows(classOf[AdminOperationException], () => adminZkClient.addPartitions(topic5, topic5Assignment, adminZkClient.getBrokerMetadatas(), 2, - Some(Map(1 -> Seq(0, 1), 2 -> Seq(0, 1, 2))))) - assertTrue(e.getMessage.contains("Unexpected existing replica assignment for topic 'new-topic5', partition id 0 is missing")) + /** + * Test that when we supply a manual partition assignment to createTopics, it must be 0-based + * and consecutive. + */ + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testMissingPartitionsInCreateTopics(quorum: String): Unit = { + val topic6Placements = new util.HashMap[Integer, util.List[Integer]] + topic6Placements.put(1, asList(0, 1)) + topic6Placements.put(2, asList(1, 0)) + val topic7Placements = new util.HashMap[Integer, util.List[Integer]] + topic7Placements.put(2, asList(0, 1)) + topic7Placements.put(3, asList(1, 0)) + val futures = admin.createTopics(asList( + new NewTopic("new-topic6", topic6Placements), + new NewTopic("new-topic7", topic7Placements))).values() + val topic6Cause = assertThrows(classOf[ExecutionException], () => futures.get("new-topic6").get()).getCause + assertEquals(classOf[InvalidReplicaAssignmentException], topic6Cause.getClass) + assertTrue(topic6Cause.getMessage.contains("partitions should be a consecutive 0-based integer sequence"), + "Unexpected error message: " + topic6Cause.getMessage) + val topic7Cause = assertThrows(classOf[ExecutionException], () => futures.get("new-topic7").get()).getCause + assertEquals(classOf[InvalidReplicaAssignmentException], topic7Cause.getClass) + assertTrue(topic7Cause.getMessage.contains("partitions should be a consecutive 0-based integer sequence"), + "Unexpected error message: " + topic7Cause.getMessage) } - @Test - def testIncrementPartitions(): Unit = { - adminZkClient.addPartitions(topic1, topic1Assignment, adminZkClient.getBrokerMetadatas(), 3) + /** + * Test that when we supply a manual partition assignment to createPartitions, it must contain + * enough partitions. + */ + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testMissingPartitionsInCreatePartitions(quorum: String): Unit = { + val cause = assertThrows(classOf[ExecutionException], () => + admin.createPartitions(Collections.singletonMap(topic1, + NewPartitions.increaseTo(3, singletonList(asList(0, 1, 2))))).all().get()).getCause + assertEquals(classOf[InvalidReplicaAssignmentException], cause.getClass) + if (isKRaftTest()) { + assertTrue(cause.getMessage.contains("Attempted to add 2 additional partition(s), but only 1 assignment(s) " + + "were specified."), "Unexpected error message: " + cause.getMessage) + } else { + assertTrue(cause.getMessage.contains("Increasing the number of partitions by 2 but 1 assignments provided."), + "Unexpected error message: " + cause.getMessage) + } + if (!isKRaftTest()) { + // In ZK mode, test the raw AdminZkClient method as well. + val e = assertThrows(classOf[AdminOperationException], () => adminZkClient.addPartitions( + topic5, topic5Assignment, adminZkClient.getBrokerMetadatas(), 2, + Some(Map(1 -> Seq(0, 1), 2 -> Seq(0, 1, 2))))) + assertTrue(e.getMessage.contains("Unexpected existing replica assignment for topic 'new-topic5', partition " + + "id 0 is missing")) + } + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testIncrementPartitions(quorum: String): Unit = { + admin.createPartitions(Collections.singletonMap(topic1, NewPartitions.increaseTo(3))).all().get() + // wait until leader is elected - val leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 1) - val leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 2) - val leader1FromZk = zkClient.getLeaderForPartition(new TopicPartition(topic1, 1)).get - val leader2FromZk = zkClient.getLeaderForPartition(new TopicPartition(topic1, 2)).get - assertEquals(leader1, leader1FromZk) - assertEquals(leader2, leader2FromZk) + waitUntilLeaderIsElectedOrChangedWithAdmin(admin, topic1, 1) + waitUntilLeaderIsElectedOrChangedWithAdmin(admin, topic1, 2) // read metadata from a broker and verify the new topic partitions exist - TestUtils.waitForPartitionMetadata(servers, topic1, 1) - TestUtils.waitForPartitionMetadata(servers, topic1, 2) + TestUtils.waitForPartitionMetadata(brokers, topic1, 1) + TestUtils.waitForPartitionMetadata(brokers, topic1, 2) val response = connectAndReceive[MetadataResponse]( new MetadataRequest.Builder(Seq(topic1).asJava, false).build) assertEquals(1, response.topicMetadata.size) @@ -102,22 +161,21 @@ class AddPartitionsTest extends BaseRequestTest { } } - @Test - def testManualAssignmentOfReplicas(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testManualAssignmentOfReplicas(quorum: String): Unit = { // Add 2 partitions - adminZkClient.addPartitions(topic2, topic2Assignment, adminZkClient.getBrokerMetadatas(), 3, - Some(Map(0 -> Seq(1, 2), 1 -> Seq(0, 1), 2 -> Seq(2, 3)))) + admin.createPartitions(Collections.singletonMap(topic2, NewPartitions.increaseTo(3, + asList(asList(0, 1), asList(2, 3))))).all().get() // wait until leader is elected - val leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 1) - val leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 2) - val leader1FromZk = zkClient.getLeaderForPartition(new TopicPartition(topic2, 1)).get - val leader2FromZk = zkClient.getLeaderForPartition(new TopicPartition(topic2, 2)).get - assertEquals(leader1, leader1FromZk) - assertEquals(leader2, leader2FromZk) + val leader1 = waitUntilLeaderIsElectedOrChangedWithAdmin(admin, topic2, 1) + val leader2 = waitUntilLeaderIsElectedOrChangedWithAdmin(admin, topic2, 2) // read metadata from a broker and verify the new topic partitions exist - TestUtils.waitForPartitionMetadata(servers, topic2, 1) - TestUtils.waitForPartitionMetadata(servers, topic2, 2) + val partition1Metadata = TestUtils.waitForPartitionMetadata(brokers, topic2, 1) + assertEquals(leader1, partition1Metadata.leader()) + val partition2Metadata = TestUtils.waitForPartitionMetadata(brokers, topic2, 2) + assertEquals(leader2, partition2Metadata.leader()) val response = connectAndReceive[MetadataResponse]( new MetadataRequest.Builder(Seq(topic2).asJava, false).build) assertEquals(1, response.topicMetadata.size) @@ -132,17 +190,18 @@ class AddPartitionsTest extends BaseRequestTest { assertEquals(Set(0, 1), replicas.asScala.toSet) } - @Test - def testReplicaPlacementAllServers(): Unit = { - adminZkClient.addPartitions(topic3, topic3Assignment, adminZkClient.getBrokerMetadatas(), 7) + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk")) // TODO: add kraft support + def testReplicaPlacementAllServers(quorum: String): Unit = { + admin.createPartitions(Collections.singletonMap(topic3, NewPartitions.increaseTo(7))).all().get() // read metadata from a broker and verify the new topic partitions exist - TestUtils.waitForPartitionMetadata(servers, topic3, 1) - TestUtils.waitForPartitionMetadata(servers, topic3, 2) - TestUtils.waitForPartitionMetadata(servers, topic3, 3) - TestUtils.waitForPartitionMetadata(servers, topic3, 4) - TestUtils.waitForPartitionMetadata(servers, topic3, 5) - TestUtils.waitForPartitionMetadata(servers, topic3, 6) + TestUtils.waitForPartitionMetadata(brokers, topic3, 1) + TestUtils.waitForPartitionMetadata(brokers, topic3, 2) + TestUtils.waitForPartitionMetadata(brokers, topic3, 3) + TestUtils.waitForPartitionMetadata(brokers, topic3, 4) + TestUtils.waitForPartitionMetadata(brokers, topic3, 5) + TestUtils.waitForPartitionMetadata(brokers, topic3, 6) val response = connectAndReceive[MetadataResponse]( new MetadataRequest.Builder(Seq(topic3).asJava, false).build) @@ -157,13 +216,14 @@ class AddPartitionsTest extends BaseRequestTest { validateLeaderAndReplicas(topicMetadata, 6, 0, Set(0, 1, 2, 3)) } - @Test - def testReplicaPlacementPartialServers(): Unit = { - adminZkClient.addPartitions(topic2, topic2Assignment, adminZkClient.getBrokerMetadatas(), 3) + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk")) // TODO: add kraft support + def testReplicaPlacementPartialServers(quorum: String): Unit = { + admin.createPartitions(Collections.singletonMap(topic2, NewPartitions.increaseTo(3))).all().get() // read metadata from a broker and verify the new topic partitions exist - TestUtils.waitForPartitionMetadata(servers, topic2, 1) - TestUtils.waitForPartitionMetadata(servers, topic2, 2) + TestUtils.waitForPartitionMetadata(brokers, topic2, 1) + TestUtils.waitForPartitionMetadata(brokers, topic2, 2) val response = connectAndReceive[MetadataResponse]( new MetadataRequest.Builder(Seq(topic2).asJava, false).build) diff --git a/core/src/test/scala/unit/kafka/admin/ConsumerGroupServiceTest.scala b/core/src/test/scala/unit/kafka/admin/ConsumerGroupServiceTest.scala index 76a3855a87274..44b241a7ed3c6 100644 --- a/core/src/test/scala/unit/kafka/admin/ConsumerGroupServiceTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConsumerGroupServiceTest.scala @@ -49,8 +49,8 @@ class ConsumerGroupServiceTest { when(admin.describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(group)), any())) .thenReturn(describeGroupsResult(ConsumerGroupState.STABLE)) - when(admin.listConsumerGroupOffsets(ArgumentMatchers.eq(group), any())) - .thenReturn(listGroupOffsetsResult) + when(admin.listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec), any())) + .thenReturn(listGroupOffsetsResult(group)) when(admin.listOffsets(offsetsArgMatcher, any())) .thenReturn(listOffsetsResult) @@ -60,7 +60,7 @@ class ConsumerGroupServiceTest { assertEquals(topicPartitions.size, assignments.get.size) verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(group)), any()) - verify(admin, times(1)).listConsumerGroupOffsets(ArgumentMatchers.eq(group), any()) + verify(admin, times(1)).listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec), any()) verify(admin, times(1)).listOffsets(offsetsArgMatcher, any()) } @@ -112,8 +112,10 @@ class ConsumerGroupServiceTest { future.complete(consumerGroupDescription) when(admin.describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(group)), any())) .thenReturn(new DescribeConsumerGroupsResult(Collections.singletonMap(group, future))) - when(admin.listConsumerGroupOffsets(ArgumentMatchers.eq(group), any())) - .thenReturn(AdminClientTestUtils.listConsumerGroupOffsetsResult(commitedOffsets)) + when(admin.listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec), any())) + .thenReturn( + AdminClientTestUtils.listConsumerGroupOffsetsResult( + Collections.singletonMap(group, commitedOffsets))) when(admin.listOffsets( ArgumentMatchers.argThat(offsetsArgMatcher(assignedTopicPartitions)), any() @@ -142,7 +144,7 @@ class ConsumerGroupServiceTest { assertEquals(expectedOffsets, returnedOffsets) verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(group)), any()) - verify(admin, times(1)).listConsumerGroupOffsets(ArgumentMatchers.eq(group), any()) + verify(admin, times(1)).listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec), any()) verify(admin, times(1)).listOffsets(ArgumentMatchers.argThat(offsetsArgMatcher(assignedTopicPartitions)), any()) verify(admin, times(1)).listOffsets(ArgumentMatchers.argThat(offsetsArgMatcher(unassignedTopicPartitions)), any()) } @@ -192,9 +194,9 @@ class ConsumerGroupServiceTest { new DescribeConsumerGroupsResult(Collections.singletonMap(group, future)) } - private def listGroupOffsetsResult: ListConsumerGroupOffsetsResult = { + private def listGroupOffsetsResult(groupId: String): ListConsumerGroupOffsetsResult = { val offsets = topicPartitions.map(_ -> new OffsetAndMetadata(100)).toMap.asJava - AdminClientTestUtils.listConsumerGroupOffsetsResult(offsets) + AdminClientTestUtils.listConsumerGroupOffsetsResult(Map(groupId -> offsets).asJava) } private def offsetsArgMatcher: util.Map[TopicPartition, OffsetSpec] = { @@ -217,4 +219,8 @@ class ConsumerGroupServiceTest { }.toMap AdminClientTestUtils.describeTopicsResult(topicDescriptions.asJava) } + + private def listConsumerGroupOffsetsSpec: util.Map[String, ListConsumerGroupOffsetsSpec] = { + Collections.singletonMap(group, new ListConsumerGroupOffsetsSpec()) + } } diff --git a/core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala b/core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala index ac715d217bd31..3becc7b330e9e 100644 --- a/core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/FeatureCommandTest.scala @@ -17,94 +17,311 @@ package kafka.admin -import kafka.server.{BaseRequestTest, KafkaConfig, KafkaServer} -import kafka.utils.TestUtils -import kafka.utils.TestUtils.waitUntilTrue -import org.apache.kafka.common.feature.{Features, SupportedVersionRange} +import kafka.api.IntegrationTestHarness +import kafka.server.KafkaConfig +import kafka.tools.TerseFailure +import kafka.utils.{TestInfoUtils, TestUtils} +import net.sourceforge.argparse4j.inf.Namespace +import org.apache.kafka.clients.admin.FeatureUpdate.UpgradeType.{SAFE_DOWNGRADE, UNSAFE_DOWNGRADE} +import org.apache.kafka.clients.admin.MockAdminClient import org.apache.kafka.common.utils.Utils -import java.util.Properties - -import org.apache.kafka.server.common.MetadataVersion.IBP_2_7_IV0 -import org.junit.jupiter.api.Assertions.assertTrue +import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.common.MetadataVersion.{IBP_3_3_IV0, IBP_3_3_IV1, IBP_3_3_IV2, IBP_3_3_IV3} +import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} import org.junit.jupiter.api.Test +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource + +import java.io.{ByteArrayOutputStream, PrintStream} +import java.{lang, util} +import java.util.Collections.{emptyMap, singletonMap} +import scala.jdk.CollectionConverters._ + +case class FeatureCommandTestEnv(admin: MockAdminClient = null) extends AutoCloseable { + val stream = new ByteArrayOutputStream() + val out = new PrintStream(stream) + + override def close(): Unit = { + Utils.closeAll(stream, out) + Utils.closeQuietly(admin, "admin") + } + + def outputWithoutEpoch(): String = { + val lines = stream.toString.split(String.format("%n")) + lines.map { line => + val pos = line.indexOf("Epoch: ") + if (pos > 0) { + line.substring(0, pos) + } else { + line + } + }.mkString(String.format("%n")) + } +} + +class FeatureCommandTest extends IntegrationTestHarness { + override def brokerCount: Int = 1 + + override protected def metadataVersion: MetadataVersion = IBP_3_3_IV1 + + serverConfig.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, metadataVersion.toString) + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk")) + def testDescribeWithZk(quorum: String): Unit = { + TestUtils.resource(FeatureCommandTestEnv()) { env => + assertEquals(0, FeatureCommand.mainNoExit( + Array("--bootstrap-server", bootstrapServers(), "describe"), env.out)) + assertEquals("", env.outputWithoutEpoch()) + } + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("kraft")) + def testDescribeWithKRaft(quorum: String): Unit = { + TestUtils.resource(FeatureCommandTestEnv()) { env => + assertEquals(0, FeatureCommand.mainNoExit( + Array("--bootstrap-server", bootstrapServers(), "describe"), env.out)) + assertEquals(String.format( + "Feature: metadata.version\tSupportedMinVersion: 3.0-IV1\t" + + "SupportedMaxVersion: 3.3-IV3\tFinalizedVersionLevel: 3.3-IV1\t"), + env.outputWithoutEpoch()) + } + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk")) + def testUpgradeMetadataVersionWithZk(quorum: String): Unit = { + TestUtils.resource(FeatureCommandTestEnv()) { env => + assertEquals(1, FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), + "upgrade", "--metadata", "3.3-IV2"), env.out)) + assertEquals("Could not upgrade metadata.version to 6. Could not apply finalized feature " + + "update because the provided feature is not supported.", env.outputWithoutEpoch()) + } + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("kraft")) + def testUpgradeMetadataVersionWithKraft(quorum: String): Unit = { + TestUtils.resource(FeatureCommandTestEnv()) { env => + assertEquals(0, FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), + "upgrade", "--feature", "metadata.version=5"), env.out)) + assertEquals("metadata.version was upgraded to 5.", env.outputWithoutEpoch()) + } + TestUtils.resource(FeatureCommandTestEnv()) { env => + assertEquals(0, FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), + "upgrade", "--metadata", "3.3-IV2"), env.out)) + assertEquals("metadata.version was upgraded to 6.", env.outputWithoutEpoch()) + } + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk")) + def testDowngradeMetadataVersionWithZk(quorum: String): Unit = { + TestUtils.resource(FeatureCommandTestEnv()) { env => + assertEquals(1, FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), + "disable", "--feature", "metadata.version"), env.out)) + assertEquals("Could not disable metadata.version. Can not delete non-existing finalized feature.", + env.outputWithoutEpoch()) + } + TestUtils.resource(FeatureCommandTestEnv()) { env => + assertEquals(1, FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), + "downgrade", "--metadata", "3.3-IV0"), env.out)) + assertEquals("Could not downgrade metadata.version to 4. Could not apply finalized feature " + + "update because the provided feature is not supported.", env.outputWithoutEpoch()) + } + TestUtils.resource(FeatureCommandTestEnv()) { env => + assertEquals(1, FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), + "downgrade", "--unsafe", "--metadata", "3.3-IV0"), env.out)) + assertEquals("Could not downgrade metadata.version to 4. Could not apply finalized feature " + + "update because the provided feature is not supported.", env.outputWithoutEpoch()) + } + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("kraft")) + def testDowngradeMetadataVersionWithKRaft(quorum: String): Unit = { + TestUtils.resource(FeatureCommandTestEnv()) { env => + assertEquals(1, FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), + "disable", "--feature", "metadata.version"), env.out)) + assertEquals("Could not disable metadata.version. Invalid update version 0 for feature " + + "metadata.version. Local controller 1000 only supports versions 1-7", env.outputWithoutEpoch()) + } + TestUtils.resource(FeatureCommandTestEnv()) { env => + assertEquals(1, FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), + "downgrade", "--metadata", "3.3-IV0"), env.out)) + assertEquals("Could not downgrade metadata.version to 4. Invalid metadata.version 4. " + + "Refusing to perform the requested downgrade because it might delete metadata information. " + + "Retry using UNSAFE_DOWNGRADE if you want to force the downgrade to proceed.", env.outputWithoutEpoch()) + } + TestUtils.resource(FeatureCommandTestEnv()) { env => + assertEquals(1, FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), + "downgrade", "--unsafe", "--metadata", "3.3-IV0"), env.out)) + assertEquals("Could not downgrade metadata.version to 4. Invalid metadata.version 4. " + + "Unsafe metadata downgrade is not supported in this version.", env.outputWithoutEpoch()) + } + } +} + +class FeatureCommandUnitTest { + @Test + def testLevelToString(): Unit = { + assertEquals("5", FeatureCommand.levelToString("foo.bar", 5.toShort)) + assertEquals("3.3-IV0", + FeatureCommand.levelToString(MetadataVersion.FEATURE_NAME, IBP_3_3_IV0.featureLevel())) + } + + @Test + def testMetadataVersionsToString(): Unit = { + assertEquals("3.3-IV0, 3.3-IV1, 3.3-IV2, 3.3-IV3", + FeatureCommand.metadataVersionsToString(IBP_3_3_IV0, IBP_3_3_IV3)) + } + + @Test + def testdowngradeType(): Unit = { + assertEquals(SAFE_DOWNGRADE, FeatureCommand.downgradeType( + new Namespace(singletonMap("unsafe", java.lang.Boolean.valueOf(false))))) + assertEquals(UNSAFE_DOWNGRADE, FeatureCommand.downgradeType( + new Namespace(singletonMap("unsafe", java.lang.Boolean.valueOf(true))))) + assertEquals(SAFE_DOWNGRADE, FeatureCommand.downgradeType(new Namespace(emptyMap()))) + } + + @Test + def testParseNameAndLevel(): Unit = { + assertEquals(("foo.bar", 5.toShort), FeatureCommand.parseNameAndLevel("foo.bar=5")) + assertEquals(("quux", 0.toShort), FeatureCommand.parseNameAndLevel(" quux=0")) + assertEquals("Can't parse feature=level string baaz: equals sign not found.", + assertThrows(classOf[TerseFailure], + () => FeatureCommand.parseNameAndLevel("baaz")).getMessage) + assertEquals("Can't parse feature=level string w=tf: unable to parse tf as a short.", + assertThrows(classOf[TerseFailure], + () => FeatureCommand.parseNameAndLevel("w=tf")).getMessage) + } + + def buildAdminClient1(): MockAdminClient = { + new MockAdminClient.Builder(). + minSupportedFeatureLevels(Map( + MetadataVersion.FEATURE_NAME -> lang.Short.valueOf(IBP_3_3_IV0.featureLevel()), + "foo.bar" -> lang.Short.valueOf(0.toShort) + ).asJava). + featureLevels(Map( + MetadataVersion.FEATURE_NAME -> lang.Short.valueOf(IBP_3_3_IV2.featureLevel()), + "foo.bar" -> lang.Short.valueOf(5.toShort) + ).asJava). + maxSupportedFeatureLevels(Map( + MetadataVersion.FEATURE_NAME -> lang.Short.valueOf(IBP_3_3_IV3.featureLevel()), + "foo.bar" -> lang.Short.valueOf(10.toShort) + ).asJava). + build() + } + + @Test + def testHandleDescribe(): Unit = { + TestUtils.resource(FeatureCommandTestEnv(buildAdminClient1())) { env => + FeatureCommand.handleDescribe(env.out, env.admin) + assertEquals(String.format( + "Feature: foo.bar\tSupportedMinVersion: 0\tSupportedMaxVersion: 10\tFinalizedVersionLevel: 5\tEpoch: 123%n" + + "Feature: metadata.version\tSupportedMinVersion: 3.3-IV0\tSupportedMaxVersion: 3.3-IV3\tFinalizedVersionLevel: 3.3-IV2\tEpoch: 123%n"), + env.stream.toString) + } + } + + @Test + def testHandleUpgrade(): Unit = { + TestUtils.resource(FeatureCommandTestEnv(buildAdminClient1())) { env => + assertEquals("1 out of 2 operation(s) failed.", + assertThrows(classOf[TerseFailure], () => + FeatureCommand.handleUpgrade(env.out, new Namespace(Map( + "metadata" -> "3.3-IV1", + "feature" -> util.Arrays.asList("foo.bar=6") + ).asJava), env.admin)).getMessage) + assertEquals(String.format( + "foo.bar was upgraded to 6.%n" + + "Could not upgrade metadata.version to 5. Can't upgrade to lower version.%n"), + env.stream.toString) + } + } + + @Test + def testHandleUpgradeDryRun(): Unit = { + TestUtils.resource(FeatureCommandTestEnv(buildAdminClient1())) { env => + assertEquals("1 out of 2 operation(s) failed.", + assertThrows(classOf[TerseFailure], () => + FeatureCommand.handleUpgrade(env.out, new Namespace(Map( + "metadata" -> "3.3-IV1", + "feature" -> util.Arrays.asList("foo.bar=6"), + "dry-run" -> java.lang.Boolean.valueOf(true) + ).asJava), env.admin)).getMessage) + assertEquals(String.format( + "foo.bar can be upgraded to 6.%n" + + "Can not upgrade metadata.version to 5. Can't upgrade to lower version.%n"), + env.stream.toString) + } + } + + @Test + def testHandleDowngrade(): Unit = { + TestUtils.resource(FeatureCommandTestEnv(buildAdminClient1())) { env => + assertEquals("1 out of 2 operation(s) failed.", + assertThrows(classOf[TerseFailure], () => + FeatureCommand.handleDowngrade(env.out, new Namespace(Map( + "metadata" -> "3.3-IV3", + "feature" -> util.Arrays.asList("foo.bar=1") + ).asJava), env.admin)).getMessage) + assertEquals(String.format( + "foo.bar was downgraded to 1.%n" + + "Could not downgrade metadata.version to 7. Can't downgrade to newer version.%n"), + env.stream.toString) + } + } + + @Test + def testHandleDowngradeDryRun(): Unit = { + TestUtils.resource(FeatureCommandTestEnv(buildAdminClient1())) { env => + assertEquals("1 out of 2 operation(s) failed.", + assertThrows(classOf[TerseFailure], () => + FeatureCommand.handleDowngrade(env.out, new Namespace(Map( + "metadata" -> "3.3-IV3", + "feature" -> util.Arrays.asList("foo.bar=1"), + "dry-run" -> java.lang.Boolean.valueOf(true) + ).asJava), env.admin)).getMessage) + assertEquals(String.format( + "foo.bar can be downgraded to 1.%n" + + "Can not downgrade metadata.version to 7. Can't downgrade to newer version.%n"), + env.stream.toString) + } + } + + @Test + def testHandleDisable(): Unit = { + TestUtils.resource(FeatureCommandTestEnv(buildAdminClient1())) { env => + assertEquals("1 out of 3 operation(s) failed.", + assertThrows(classOf[TerseFailure], () => + FeatureCommand.handleDisable(env.out, new Namespace(Map[String, AnyRef]( + "feature" -> util.Arrays.asList("foo.bar", "metadata.version", "quux") + ).asJava), env.admin)).getMessage) + assertEquals(String.format( + "foo.bar was disabled.%n" + + "Could not disable metadata.version. Can't downgrade below 4%n" + + "quux was disabled.%n"), + env.stream.toString) + } + } -class FeatureCommandTest extends BaseRequestTest { - override def brokerCount: Int = 3 - - override def brokerPropertyOverrides(props: Properties): Unit = { - props.put(KafkaConfig.InterBrokerProtocolVersionProp, IBP_2_7_IV0.toString) - } - - private val defaultSupportedFeatures: Features[SupportedVersionRange] = - Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), - Utils.mkEntry("feature_2", new SupportedVersionRange(1, 5)))) - - private def updateSupportedFeatures(features: Features[SupportedVersionRange], - targetServers: Set[KafkaServer]): Unit = { - targetServers.foreach(s => { - s.brokerFeatures.setSupportedFeatures(features) - s.zkClient.updateBrokerInfo(s.createBrokerInfo) - }) - - // Wait until updates to all BrokerZNode supported features propagate to the controller. - val brokerIds = targetServers.map(s => s.config.brokerId) - waitUntilTrue( - () => servers.exists(s => { - if (s.kafkaController.isActive) { - s.kafkaController.controllerContext.liveOrShuttingDownBrokers - .filter(b => brokerIds.contains(b.id)) - .forall(b => { - b.features.equals(features) - }) - } else { - false - } - }), - "Controller did not get broker updates") - } - - private def updateSupportedFeaturesInAllBrokers(features: Features[SupportedVersionRange]): Unit = { - updateSupportedFeatures(features, Set[KafkaServer]() ++ servers) - } - - /** - * Tests if the FeatureApis#describeFeatures API works as expected when describing features before and - * after upgrading features. - */ @Test - def testDescribeFeaturesSuccess(): Unit = { - updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures) - - val initialDescribeOutput = TestUtils.grabConsoleOutput(FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), "describe"))) - val expectedInitialDescribeOutputs = Seq( - "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedVersionLevel: -", - "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedVersionLevel: -" - ) - - expectedInitialDescribeOutputs.foreach { expectedOutput => - assertTrue(initialDescribeOutput.contains(expectedOutput)) - } - - FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), "upgrade", - "--feature", "feature_1", "--version", "3", "--feature", "feature_2", "--version", "5")) - val upgradeDescribeOutput = TestUtils.grabConsoleOutput(FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), "describe"))) - val expectedUpgradeDescribeOutput = Seq( - "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedVersionLevel: 3", - "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedVersionLevel: 5" - ) - expectedUpgradeDescribeOutput.foreach { expectedOutput => - assertTrue(upgradeDescribeOutput.contains(expectedOutput)) - } - - FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), "downgrade", - "--feature", "feature_1", "--version", "2", "--feature", "feature_2", "--version", "2")) - val downgradeDescribeOutput = TestUtils.grabConsoleOutput(FeatureCommand.mainNoExit(Array("--bootstrap-server", bootstrapServers(), "describe"))) - val expectedFinalDescribeOutput = Seq( - "Feature: feature_1\tSupportedMinVersion: 1\tSupportedMaxVersion: 3\tFinalizedVersionLevel: 2", - "Feature: feature_2\tSupportedMinVersion: 1\tSupportedMaxVersion: 5\tFinalizedVersionLevel: 2" - ) - expectedFinalDescribeOutput.foreach { expectedOutput => - assertTrue(downgradeDescribeOutput.contains(expectedOutput)) + def testHandleDisableDryRun(): Unit = { + TestUtils.resource(FeatureCommandTestEnv(buildAdminClient1())) { env => + assertEquals("1 out of 3 operation(s) failed.", + assertThrows(classOf[TerseFailure], () => + FeatureCommand.handleDisable(env.out, new Namespace(Map[String, AnyRef]( + "feature" -> util.Arrays.asList("foo.bar", "metadata.version", "quux"), + "dry-run" -> java.lang.Boolean.valueOf(true) + ).asJava), env.admin)).getMessage) + assertEquals(String.format( + "foo.bar can be disabled.%n" + + "Can not disable metadata.version. Can't downgrade below 4%n" + + "quux can be disabled.%n"), + env.stream.toString) } } } diff --git a/core/src/test/scala/unit/kafka/admin/MetadataQuorumCommandTest.scala b/core/src/test/scala/unit/kafka/admin/MetadataQuorumCommandTest.scala new file mode 100644 index 0000000000000..24b6616cb1edb --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/MetadataQuorumCommandTest.scala @@ -0,0 +1,192 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.admin + +import kafka.test.ClusterInstance +import kafka.test.annotation.{ClusterTest, ClusterTestDefaults, ClusterTests, Type} +import kafka.test.junit.ClusterTestExtensions +import kafka.utils.TestUtils +import org.apache.kafka.common.errors.UnsupportedVersionException +import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue} +import org.junit.jupiter.api.{Tag, Test} +import org.junit.jupiter.api.extension.ExtendWith + +import java.util.concurrent.ExecutionException + +@ExtendWith(value = Array(classOf[ClusterTestExtensions])) +@ClusterTestDefaults(clusterType = Type.KRAFT) +@Tag("integration") +class MetadataQuorumCommandTest(cluster: ClusterInstance) { + + /** + * 1. The same number of broker controllers + * 2. More brokers than controllers + * 3. Fewer brokers than controllers + */ + @ClusterTests( + Array( + new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 3), + new ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 3), + new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 2), + new ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 2), + new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 3), + new ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 3) + )) + def testDescribeQuorumReplicationSuccessful(): Unit = { + cluster.waitForReadyBrokers() + val describeOutput = TestUtils.grabConsoleOutput( + MetadataQuorumCommand.mainNoExit( + Array("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication")) + ) + + val leaderPattern = """\d+\s+\d+\s+\d+\s+\d+\s+[-]?\d+\s+Leader\s+""".r + val followerPattern = """\d+\s+\d+\s+\d+\s+\d+\s+[-]?\d+\s+Follower\s+""".r + val observerPattern = """\d+\s+\d+\s+\d+\s+\d+\s+[-]?\d+\s+Observer\s+""".r + val outputs = describeOutput.split("\n").tail + if (cluster.config().clusterType() == Type.CO_KRAFT) { + assertEquals(Math.max(cluster.config().numControllers(), cluster.config().numBrokers()), outputs.length) + } else { + assertEquals(cluster.config().numBrokers() + cluster.config().numControllers(), outputs.length) + } + // `matches` is not supported in scala 2.12, use `findFirstIn` instead. + assertTrue(leaderPattern.findFirstIn(outputs.head).nonEmpty) + assertEquals(1, outputs.count(leaderPattern.findFirstIn(_).nonEmpty)) + assertEquals(cluster.config().numControllers() - 1, outputs.count(followerPattern.findFirstIn(_).nonEmpty)) + + if (cluster.config().clusterType() == Type.CO_KRAFT) { + assertEquals(Math.max(0, cluster.config().numBrokers() - cluster.config().numControllers()), outputs.count(observerPattern.findFirstIn(_).nonEmpty)) + } else { + assertEquals(cluster.config().numBrokers(), outputs.count(observerPattern.findFirstIn(_).nonEmpty)) + } + } + + /** + * 1. The same number of broker controllers + * 2. More brokers than controllers + * 3. Fewer brokers than controllers + */ + @ClusterTests( + Array( + new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 3), + new ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 3), + new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 2), + new ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 2), + new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 3), + new ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 3) + )) + def testDescribeQuorumStatusSuccessful(): Unit = { + cluster.waitForReadyBrokers() + val describeOutput = TestUtils.grabConsoleOutput( + MetadataQuorumCommand.mainNoExit(Array("--bootstrap-server", cluster.bootstrapServers(), "describe", "--status")) + ) + val outputs = describeOutput.split("\n") + + assertTrue("""ClusterId:\s+\S{22}""".r.findFirstIn(outputs(0)).nonEmpty) + assertTrue("""LeaderId:\s+\d+""".r.findFirstIn(outputs(1)).nonEmpty) + assertTrue("""LeaderEpoch:\s+\d+""".r.findFirstIn(outputs(2)).nonEmpty) + // HighWatermark may be -1 + assertTrue("""HighWatermark:\s+[-]?\d+""".r.findFirstIn(outputs(3)).nonEmpty) + assertTrue("""MaxFollowerLag:\s+\d+""".r.findFirstIn(outputs(4)).nonEmpty) + assertTrue("""MaxFollowerLagTimeMs:\s+[-]?\d+""".r.findFirstIn(outputs(5)).nonEmpty) + assertTrue("""CurrentVoters:\s+\[\d+(,\d+)*\]""".r.findFirstIn(outputs(6)).nonEmpty) + + // There are no observers if we have fewer brokers than controllers + if (cluster.config().clusterType() == Type.CO_KRAFT + && cluster.config().numBrokers() <= cluster.config().numControllers()) { + assertTrue("""CurrentObservers:\s+\[\]""".r.findFirstIn(outputs(7)).nonEmpty) + } else { + assertTrue("""CurrentObservers:\s+\[\d+(,\d+)*\]""".r.findFirstIn(outputs(7)).nonEmpty) + } + } + + @ClusterTests( + Array(new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 1, controllers = 1), + new ClusterTest(clusterType = Type.KRAFT, brokers = 1, controllers = 1))) + def testOnlyOneBrokerAndOneController(): Unit = { + val statusOutput = TestUtils.grabConsoleOutput( + MetadataQuorumCommand.mainNoExit(Array("--bootstrap-server", cluster.bootstrapServers(), "describe", "--status")) + ) + assertEquals("MaxFollowerLag: 0", statusOutput.split("\n")(4)) + assertEquals("MaxFollowerLagTimeMs: 0", statusOutput.split("\n")(5)) + + val replicationOutput = TestUtils.grabConsoleOutput( + MetadataQuorumCommand.mainNoExit(Array("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication")) + ) + assertEquals("0", replicationOutput.split("\n")(1).split("\\s+")(2)) + } + + @ClusterTest(clusterType = Type.ZK, brokers = 3) + def testDescribeQuorumInZkMode(): Unit = { + assertTrue( + assertThrows( + classOf[ExecutionException], + () => + MetadataQuorumCommand.mainNoExit( + Array("--bootstrap-server", cluster.bootstrapServers(), "describe", "--status")) + ).getCause.isInstanceOf[UnsupportedVersionException] + ) + assertTrue( + assertThrows( + classOf[ExecutionException], + () => + MetadataQuorumCommand.mainNoExit( + Array("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication")) + ).getCause.isInstanceOf[UnsupportedVersionException] + ) + } +} + +class MetadataQuorumCommandErrorTest { + + @Test + def testPropertiesFileDoesNotExists(): Unit = { + assertEquals(1, + MetadataQuorumCommand.mainNoExit( + Array("--bootstrap-server", "localhost:9092", "--command-config", "admin.properties", "describe"))) + assertEquals( + "Properties file admin.properties does not exists!", + TestUtils + .grabConsoleError( + MetadataQuorumCommand.mainNoExit( + Array("--bootstrap-server", "localhost:9092", "--command-config", "admin.properties", "describe"))) + .trim + ) + } + + @Test + def testDescribeOptions(): Unit = { + assertEquals(1, MetadataQuorumCommand.mainNoExit(Array("--bootstrap-server", "localhost:9092", "describe"))) + assertEquals( + "One of --status or --replication must be specified with describe sub-command", + TestUtils + .grabConsoleError(MetadataQuorumCommand.mainNoExit(Array("--bootstrap-server", "localhost:9092", "describe"))) + .trim + ) + + assertEquals(1, + MetadataQuorumCommand.mainNoExit( + Array("--bootstrap-server", "localhost:9092", "describe", "--status", "--replication"))) + assertEquals( + "Only one of --status or --replication should be specified with describe sub-command", + TestUtils + .grabConsoleError( + MetadataQuorumCommand.mainNoExit( + Array("--bootstrap-server", "localhost:9092", "describe", "--status", "--replication"))) + .trim + ) + } +} diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 65a6cdadf433a..948abc6c3b87f 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -46,7 +46,7 @@ import java.nio.ByteBuffer import java.util.Optional import java.util.concurrent.{CountDownLatch, Semaphore} import kafka.server.epoch.LeaderEpochFileCache -import kafka.server.metadata.KRaftMetadataCache +import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache} import org.apache.kafka.clients.ClientResponse import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.replica.ClientMetadata @@ -55,6 +55,8 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0 import org.apache.kafka.server.metrics.KafkaYammerMetrics +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ @@ -238,6 +240,47 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(None, partition.futureLog) } + @Test + def testReplicaFetchToFollower(): Unit = { + val controllerEpoch = 3 + val followerId = brokerId + 1 + val leaderId = brokerId + 2 + val replicas = List[Integer](brokerId, followerId, leaderId).asJava + val isr = List[Integer](brokerId, followerId, leaderId).asJava + val leaderEpoch = 8 + val partitionEpoch = 1 + + assertTrue(partition.makeFollower(new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(leaderId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setPartitionEpoch(partitionEpoch) + .setReplicas(replicas) + .setIsNew(true), + offsetCheckpoints, None + )) + + def assertFetchFromReplicaFails[T <: ApiException]( + expectedExceptionClass: Class[T], + leaderEpoch: Option[Int] + ): Unit = { + assertThrows(expectedExceptionClass, () => { + fetchFollower( + partition, + replicaId = followerId, + fetchOffset = 0L, + leaderEpoch = leaderEpoch + ) + }) + } + + assertFetchFromReplicaFails(classOf[NotLeaderOrFollowerException], None) + assertFetchFromReplicaFails(classOf[NotLeaderOrFollowerException], Some(leaderEpoch)) + assertFetchFromReplicaFails(classOf[UnknownLeaderEpochException], Some(leaderEpoch + 1)) + assertFetchFromReplicaFails(classOf[FencedLeaderEpochException], Some(leaderEpoch - 1)) + } + @Test def testFetchFromUnrecognizedFollower(): Unit = { val controllerEpoch = 3 @@ -1334,8 +1377,11 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(alterPartitionListener.failures.get, 1) } - @Test - def testIsrNotExpandedIfReplicaIsFenced(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testIsrNotExpandedIfReplicaIsFencedOrShutdown(quorum: String): Unit = { + val kraft = quorum == "kraft" + val log = logManager.getOrCreateLog(topicPartition, topicId = None) seedLogData(log, numRecords = 10, leaderEpoch = 4) @@ -1345,7 +1391,19 @@ class PartitionTest extends AbstractPartitionTest { val replicas = List(brokerId, remoteBrokerId) val isr = Set(brokerId) - val metadataCache = mock(classOf[KRaftMetadataCache]) + val metadataCache: MetadataCache = if (kraft) mock(classOf[KRaftMetadataCache]) else mock(classOf[ZkMetadataCache]) + + // Mark the remote broker as eligible or ineligible in the metadata cache of the leader. + // When using kraft, we can make the broker ineligible by fencing it. + // In ZK mode, we must mark the broker as alive for it to be eligible. + def markRemoteReplicaEligible(eligible: Boolean): Unit = { + if (kraft) { + when(metadataCache.asInstanceOf[KRaftMetadataCache].isBrokerFenced(remoteBrokerId)).thenReturn(!eligible) + } else { + when(metadataCache.hasAliveBroker(remoteBrokerId)).thenReturn(eligible) + } + } + val partition = new Partition( topicPartition, replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, @@ -1373,6 +1431,8 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(isr, partition.partitionState.isr) assertEquals(isr, partition.partitionState.maximalIsr) + markRemoteReplicaEligible(true) + // Fetch to let the follower catch up to the log end offset and // to check if an expansion is possible. fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset) @@ -1389,7 +1449,7 @@ class PartitionTest extends AbstractPartitionTest { assertEquals(replicas.toSet, partition.partitionState.maximalIsr) assertEquals(1, alterPartitionManager.isrUpdates.size) - // Controller rejects the expansion because the broker is fenced. + // Controller rejects the expansion because the broker is fenced or offline. alterPartitionManager.failIsrUpdate(Errors.INELIGIBLE_REPLICA) // The leader reverts back to the previous ISR. @@ -1398,8 +1458,8 @@ class PartitionTest extends AbstractPartitionTest { assertFalse(partition.partitionState.isInflight) assertEquals(0, alterPartitionManager.isrUpdates.size) - // The leader eventually learns about the fenced broker. - when(metadataCache.isBrokerFenced(remoteBrokerId)).thenReturn(true) + // The leader eventually learns about the fenced or offline broker. + markRemoteReplicaEligible(false) // The follower fetches again. fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset) @@ -1410,8 +1470,8 @@ class PartitionTest extends AbstractPartitionTest { assertFalse(partition.partitionState.isInflight) assertEquals(0, alterPartitionManager.isrUpdates.size) - // The broker is eventually unfenced. - when(metadataCache.isBrokerFenced(remoteBrokerId)).thenReturn(false) + // The broker is eventually unfenced or brought back online. + markRemoteReplicaEligible(true) // The follower fetches again. fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = log.logEndOffset) diff --git a/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala index e8efa5af79356..e88bb321ad9b3 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala @@ -203,4 +203,25 @@ class ControllerContextTest { context.removeTopic(tp3.topic) assertEquals(0, context.preferredReplicaImbalanceCount) } + + @Test + def testPreferredReplicaImbalanceMetricOnConcurrentTopicDeletion(): Unit = { + val topicA = "A" + val topicB = "B" + val tpA = new TopicPartition(topicA, 0) + val tpB = new TopicPartition(topicB, 0) + context.updatePartitionFullReplicaAssignment(tpA, ReplicaAssignment(Seq(1, 2, 3))) + context.updatePartitionFullReplicaAssignment(tpB, ReplicaAssignment(Seq(1, 2, 3))) + assertEquals(0, context.preferredReplicaImbalanceCount) + + context.queueTopicDeletion(Set(topicA)) + // All partitions in topic will be marked as Offline during deletion procedure + context.putPartitionLeadershipInfo(tpA, LeaderIsrAndControllerEpoch(LeaderAndIsr(LeaderAndIsr.NoLeader, List(1, 2, 3)), 0)) + assertEquals(0, context.preferredReplicaImbalanceCount) + + // Initiate topicB's topic deletion before topicA's deletion completes. + // Since topicA's delete-topic ZK node still exists, context.queueTopicDeletion will be called with Set(topicA, topicB) + context.queueTopicDeletion(Set(topicA, topicB)) + assertEquals(0, context.preferredReplicaImbalanceCount) + } } diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 57cbeafd4d033..532ff1a946e9e 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -26,11 +26,11 @@ import kafka.server.{KafkaConfig, KafkaServer, QuorumTestHarness} import kafka.utils.{LogCaptureAppender, TestUtils} import kafka.zk.{FeatureZNodeStatus, _} import org.apache.kafka.common.errors.{ControllerMovedException, StaleBrokerEpochException} -import org.apache.kafka.common.message.AlterPartitionRequestData -import org.apache.kafka.common.message.AlterPartitionResponseData +import org.apache.kafka.common.message.{AlterPartitionRequestData, AlterPartitionResponseData} import org.apache.kafka.common.metrics.KafkaMetric import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource import org.apache.kafka.common.{ElectionType, TopicPartition, Uuid} import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.server.common.MetadataVersion @@ -40,8 +40,7 @@ import org.apache.log4j.Level import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.Arguments -import org.junit.jupiter.params.provider.MethodSource +import org.junit.jupiter.params.provider.{Arguments, MethodSource} import org.mockito.Mockito.{doAnswer, spy, verify} import org.mockito.invocation.InvocationOnMock @@ -904,12 +903,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { ).asJava) ).asJava) - val future = new CompletableFuture[AlterPartitionResponseData]() - controller.eventManager.put(AlterPartitionReceived( - alterPartitionRequest, - alterPartitionVersion, - future.complete - )) + val future = alterPartitionFuture(alterPartitionRequest, alterPartitionVersion) val expectedAlterPartitionResponse = new AlterPartitionResponseData() .setTopics(Seq(new AlterPartitionResponseData.TopicData() @@ -968,12 +962,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { ).asJava) ).asJava) - val future = new CompletableFuture[AlterPartitionResponseData]() - controller.eventManager.put(AlterPartitionReceived( - alterPartitionRequest, - ApiKeys.ALTER_PARTITION.latestVersion, - future.complete - )) + val future = alterPartitionFuture(alterPartitionRequest, ApiKeys.ALTER_PARTITION.latestVersion) val expectedAlterPartitionResponse = new AlterPartitionResponseData() .setTopics(Seq(new AlterPartitionResponseData.TopicData() @@ -1002,47 +991,120 @@ class ControllerIntegrationTest extends QuorumTestHarness { val controller = getController().kafkaController val leaderIsrAndControllerEpochMap = zkClient.getTopicPartitionStates(Seq(tp)) - val newLeaderAndIsr = leaderIsrAndControllerEpochMap(tp).leaderAndIsr + val oldLeaderAndIsr = leaderIsrAndControllerEpochMap(tp).leaderAndIsr + val newIsr = List(oldLeaderAndIsr.leader) + val newPartitionEpoch = oldLeaderAndIsr.partitionEpoch + 1 val topicId = controller.controllerContext.topicIds(tp.topic) val brokerId = otherBroker.config.brokerId val brokerEpoch = controller.controllerContext.liveBrokerIdAndEpochs(otherBroker.config.brokerId) - // When re-sending the current ISR, we should not get and error or any ISR changes - val alterPartitionRequest = new AlterPartitionRequestData() - .setBrokerId(brokerId) - .setBrokerEpoch(brokerEpoch) - .setTopics(Seq(new AlterPartitionRequestData.TopicData() - .setTopicId(topicId) - .setPartitions(Seq(new AlterPartitionRequestData.PartitionData() - .setPartitionIndex(tp.partition) - .setLeaderEpoch(newLeaderAndIsr.leaderEpoch) - .setPartitionEpoch(newLeaderAndIsr.partitionEpoch) - .setNewIsr(newLeaderAndIsr.isr.map(Int.box).asJava) - .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) + def sendAndVerifyAlterPartitionResponse(requestPartitionEpoch: Int): Unit = { + val alterPartitionRequest = new AlterPartitionRequestData() + .setBrokerId(brokerId) + .setBrokerEpoch(brokerEpoch) + .setTopics(Seq(new AlterPartitionRequestData.TopicData() + .setTopicId(topicId) + .setPartitions(Seq(new AlterPartitionRequestData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderEpoch(oldLeaderAndIsr.leaderEpoch) + .setPartitionEpoch(requestPartitionEpoch) + .setNewIsr(newIsr.map(Int.box).asJava) + .setLeaderRecoveryState(oldLeaderAndIsr.leaderRecoveryState.value) + ).asJava) ).asJava) - ).asJava) - val future = new CompletableFuture[AlterPartitionResponseData]() - controller.eventManager.put(AlterPartitionReceived( - alterPartitionRequest, - AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION, - future.complete - )) + val future = alterPartitionFuture(alterPartitionRequest, AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION) - val expectedAlterPartitionResponse = new AlterPartitionResponseData() - .setTopics(Seq(new AlterPartitionResponseData.TopicData() - .setTopicId(topicId) - .setPartitions(Seq(new AlterPartitionResponseData.PartitionData() - .setPartitionIndex(tp.partition) - .setLeaderId(brokerId) - .setLeaderEpoch(newLeaderAndIsr.leaderEpoch) - .setPartitionEpoch(newLeaderAndIsr.partitionEpoch) - .setIsr(newLeaderAndIsr.isr.map(Int.box).asJava) - .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) + // When re-sending an ISR update, we should not get and error or any ISR changes + val expectedAlterPartitionResponse = new AlterPartitionResponseData() + .setTopics(Seq(new AlterPartitionResponseData.TopicData() + .setTopicId(topicId) + .setPartitions(Seq(new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderId(brokerId) + .setLeaderEpoch(oldLeaderAndIsr.leaderEpoch) + .setPartitionEpoch(newPartitionEpoch) + .setIsr(newIsr.map(Int.box).asJava) + .setLeaderRecoveryState(oldLeaderAndIsr.leaderRecoveryState.value) + ).asJava) ).asJava) + assertEquals(expectedAlterPartitionResponse, future.get(10, TimeUnit.SECONDS)) + } + + // send a request, expect the partition epoch to be incremented + sendAndVerifyAlterPartitionResponse(oldLeaderAndIsr.partitionEpoch) + + // re-send the same request with various partition epochs (less/equal/greater than the current + // epoch), expect it to succeed while the partition epoch remains the same + sendAndVerifyAlterPartitionResponse(oldLeaderAndIsr.partitionEpoch) + sendAndVerifyAlterPartitionResponse(newPartitionEpoch) + } + + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.ALTER_PARTITION) + def testShutdownBrokerNotAddedToIsr(alterPartitionVersion: Short): Unit = { + servers = makeServers(2) + val controllerId = TestUtils.waitUntilControllerElected(zkClient) + val otherBroker = servers.find(_.config.brokerId != controllerId).get + val brokerId = otherBroker.config.brokerId + val tp = new TopicPartition("t", 0) + val assignment = Map(tp.partition -> Seq(controllerId, brokerId)) + val fullIsr = List(controllerId, brokerId) + TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) + + // Shut down follower. + servers(brokerId).shutdown() + servers(brokerId).awaitShutdown() + + val controller = getController().kafkaController + val leaderIsrAndControllerEpochMap = controller.controllerContext.partitionsLeadershipInfo + val leaderAndIsr = leaderIsrAndControllerEpochMap(tp).leaderAndIsr + val topicId = controller.controllerContext.topicIds(tp.topic) + val controllerEpoch = controller.controllerContext.liveBrokerIdAndEpochs(controllerId) + + // We expect only the controller (online broker) to be in ISR + assertEquals(List(controllerId), leaderAndIsr.isr) + + val requestTopic = new AlterPartitionRequestData.TopicData() + .setPartitions(Seq(new AlterPartitionRequestData.PartitionData() + .setPartitionIndex(tp.partition) + .setLeaderEpoch(leaderAndIsr.leaderEpoch) + .setPartitionEpoch(leaderAndIsr.partitionEpoch) + .setNewIsr(fullIsr.map(Int.box).asJava) + .setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value)).asJava) + if (alterPartitionVersion > 1) requestTopic.setTopicId(topicId) else requestTopic.setTopicName(tp.topic) + + // Try to update ISR to contain the offline broker. + val alterPartitionRequest = new AlterPartitionRequestData() + .setBrokerId(controllerId) + .setBrokerEpoch(controllerEpoch) + .setTopics(Seq(requestTopic).asJava) + + val future = alterPartitionFuture(alterPartitionRequest, alterPartitionVersion) + + val expectedError = if (alterPartitionVersion > 1) Errors.INELIGIBLE_REPLICA else Errors.OPERATION_NOT_ATTEMPTED + val expectedResponseTopic = new AlterPartitionResponseData.TopicData() + .setPartitions(Seq(new AlterPartitionResponseData.PartitionData() + .setPartitionIndex(tp.partition) + .setErrorCode(expectedError.code()) + .setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value) ).asJava) + if (alterPartitionVersion > 1) expectedResponseTopic.setTopicId(topicId) else expectedResponseTopic.setTopicName(tp.topic) + // We expect an ineligble replica error response for the partition. + val expectedAlterPartitionResponse = new AlterPartitionResponseData() + .setTopics(Seq(expectedResponseTopic).asJava) + + val newLeaderIsrAndControllerEpochMap = controller.controllerContext.partitionsLeadershipInfo + val newLeaderAndIsr = newLeaderIsrAndControllerEpochMap(tp).leaderAndIsr assertEquals(expectedAlterPartitionResponse, future.get(10, TimeUnit.SECONDS)) + assertEquals(List(controllerId), newLeaderAndIsr.isr) + + // Bring replica back online. + servers(brokerId).startup() + + // Wait for broker to rejoin ISR. + TestUtils.waitUntilTrue(() => fullIsr == zkClient.getTopicPartitionState(tp).get.leaderAndIsr.isr, "Replica did not rejoin ISR.") } @Test @@ -1100,7 +1162,6 @@ class ControllerIntegrationTest extends QuorumTestHarness { assertAlterPartition( partitionError = Errors.UNKNOWN_TOPIC_ID, - topicPartition = tp, topicIdOpt = Some(Uuid.randomUuid()) ) @@ -1118,16 +1179,22 @@ class ControllerIntegrationTest extends QuorumTestHarness { assertAlterPartition( partitionError = Errors.INVALID_UPDATE_VERSION, + isr = Set(leaderId), partitionEpoch = partitionEpoch - 1 ) + assertAlterPartition( + partitionError = Errors.NOT_CONTROLLER, + partitionEpoch = partitionEpoch + 1 + ) + assertAlterPartition( partitionError = Errors.FENCED_LEADER_EPOCH, leaderEpoch = leaderEpoch - 1 ) assertAlterPartition( - partitionError = Errors.FENCED_LEADER_EPOCH, + partitionError = Errors.NOT_CONTROLLER, leaderEpoch = leaderEpoch + 1 ) @@ -1151,6 +1218,12 @@ class ControllerIntegrationTest extends QuorumTestHarness { partitionEpoch = partitionEpoch - 1 ) + assertAlterPartition( + partitionError = Errors.NOT_CONTROLLER, + leaderRecoveryState = LeaderRecoveryState.RECOVERING.value, + partitionEpoch = partitionEpoch + 1 + ) + assertAlterPartition( partitionError = Errors.FENCED_LEADER_EPOCH, leaderRecoveryState = LeaderRecoveryState.RECOVERING.value, @@ -1158,7 +1231,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { ) assertAlterPartition( - partitionError = Errors.FENCED_LEADER_EPOCH, + partitionError = Errors.NOT_CONTROLLER, leaderRecoveryState = LeaderRecoveryState.RECOVERING.value, leaderEpoch = leaderEpoch + 1 ) @@ -1257,13 +1330,18 @@ class ControllerIntegrationTest extends QuorumTestHarness { partitionEpoch = partitionEpoch - 1 ) + assertAlterPartition( + partitionError = Errors.NOT_CONTROLLER, + partitionEpoch = partitionEpoch + 1 + ) + assertAlterPartition( partitionError = Errors.FENCED_LEADER_EPOCH, leaderEpoch = leaderEpoch - 1 ) assertAlterPartition( - partitionError = Errors.FENCED_LEADER_EPOCH, + partitionError = Errors.NOT_CONTROLLER, leaderEpoch = leaderEpoch + 1 ) @@ -1281,6 +1359,12 @@ class ControllerIntegrationTest extends QuorumTestHarness { leaderRecoveryState = LeaderRecoveryState.RECOVERING.value ) + assertAlterPartition( + partitionError = Errors.NOT_CONTROLLER, + partitionEpoch = partitionEpoch + 1, + leaderRecoveryState = LeaderRecoveryState.RECOVERING.value + ) + assertAlterPartition( partitionError = Errors.FENCED_LEADER_EPOCH, leaderEpoch = leaderEpoch - 1, @@ -1288,7 +1372,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { ) assertAlterPartition( - partitionError = Errors.FENCED_LEADER_EPOCH, + partitionError = Errors.NOT_CONTROLLER, leaderEpoch = leaderEpoch + 1, leaderRecoveryState = LeaderRecoveryState.RECOVERING.value ) @@ -1322,12 +1406,7 @@ class ControllerIntegrationTest extends QuorumTestHarness { .setNewIsr(isr.toList.map(Int.box).asJava) .setLeaderRecoveryState(leaderRecoveryState)).asJava)).asJava) - val future = new CompletableFuture[AlterPartitionResponseData]() - getController().kafkaController.eventManager.put(AlterPartitionReceived( - alterPartitionRequest, - if (topicIdOpt.isDefined) AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION else 1, - future.complete - )) + val future = alterPartitionFuture(alterPartitionRequest, if (topicIdOpt.isDefined) AlterPartitionRequestData.HIGHEST_SUPPORTED_VERSION else 1) val expectedAlterPartitionResponse = if (topLevelError != Errors.NONE) { new AlterPartitionResponseData().setErrorCode(topLevelError.code) @@ -1802,4 +1881,15 @@ class ControllerIntegrationTest extends QuorumTestHarness { servers.filter(s => s.config.brokerId == controllerId).head } + private def alterPartitionFuture(alterPartitionRequest: AlterPartitionRequestData, + alterPartitionVersion: Short): CompletableFuture[AlterPartitionResponseData] = { + val future = new CompletableFuture[AlterPartitionResponseData]() + getController().kafkaController.eventManager.put(AlterPartitionReceived( + alterPartitionRequest, + alterPartitionVersion, + future.complete + )) + future + } + } 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 164c9ab1fefb3..45b99eda293cf 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -1036,6 +1036,52 @@ class GroupCoordinatorTest { assertEquals(Errors.NONE, syncGroupWithOldMemberIdResult.error) } + @Test + def staticMemberRejoinWithUpdatedSessionAndRebalanceTimeoutsButCannotPersistChange(): Unit = { + val rebalanceResult = staticMembersJoinAndRebalance(leaderInstanceId, followerInstanceId) + val joinGroupResult = staticJoinGroupWithPersistence(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, followerInstanceId, protocolType, protocolSuperset, clockAdvance = 1, 2 * DefaultSessionTimeout, 2 * DefaultRebalanceTimeout, appendRecordError = Errors.MESSAGE_TOO_LARGE) + checkJoinGroupResult(joinGroupResult, + Errors.UNKNOWN_SERVER_ERROR, + rebalanceResult.generation, + Set.empty, + Stable, + Some(protocolType)) + assertTrue(groupCoordinator.groupManager.getGroup(groupId).isDefined) + val group = groupCoordinator.groupManager.getGroup(groupId).get + group.allMemberMetadata.foreach { member => + assertEquals(member.sessionTimeoutMs, DefaultSessionTimeout) + assertEquals(member.rebalanceTimeoutMs, DefaultRebalanceTimeout) + } + } + + + @Test + def staticMemberRejoinWithUpdatedSessionAndRebalanceTimeoutsAndPersistChange(): Unit = { + val rebalanceResult = staticMembersJoinAndRebalance(leaderInstanceId, followerInstanceId) + val followerJoinGroupResult = staticJoinGroupWithPersistence(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, followerInstanceId, protocolType, protocolSuperset, clockAdvance = 1, 2 * DefaultSessionTimeout, 2 * DefaultRebalanceTimeout) + checkJoinGroupResult(followerJoinGroupResult, + Errors.NONE, + rebalanceResult.generation, + Set.empty, + Stable, + Some(protocolType)) + val leaderJoinGroupResult = staticJoinGroupWithPersistence(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, leaderInstanceId, protocolType, protocolSuperset, clockAdvance = 1, 2 * DefaultSessionTimeout, 2 * DefaultRebalanceTimeout) + checkJoinGroupResult(leaderJoinGroupResult, + Errors.NONE, + rebalanceResult.generation, + Set(leaderInstanceId, followerInstanceId), + Stable, + Some(protocolType), + leaderJoinGroupResult.leaderId, + leaderJoinGroupResult.memberId, + true) + assertTrue(groupCoordinator.groupManager.getGroup(groupId).isDefined) + val group = groupCoordinator.groupManager.getGroup(groupId).get + group.allMemberMetadata.foreach { member => + assertEquals(member.sessionTimeoutMs, 2 * DefaultSessionTimeout) + assertEquals(member.rebalanceTimeoutMs, 2 * DefaultRebalanceTimeout) + } + } @Test def staticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSelectProtocolUnchanged(): Unit = { val rebalanceResult = staticMembersJoinAndRebalance(leaderInstanceId, followerInstanceId) @@ -1263,7 +1309,6 @@ class GroupCoordinatorTest { group.transitionTo(PreparingRebalance) group.transitionTo(Empty) - // Illegal state exception shall trigger since follower id resides in pending member bucket. val expectedException = assertThrows(classOf[IllegalStateException], () => staticJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, followerInstanceId, protocolType, protocolSuperset, clockAdvance = 1)) @@ -1520,12 +1565,13 @@ class GroupCoordinatorTest { */ private def staticMembersJoinAndRebalance(leaderInstanceId: String, followerInstanceId: String, - sessionTimeout: Int = DefaultSessionTimeout): RebalanceResult = { + sessionTimeout: Int = DefaultSessionTimeout, + rebalanceTimeout: Int = DefaultRebalanceTimeout): RebalanceResult = { val leaderResponseFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, - protocolSuperset, Some(leaderInstanceId), sessionTimeout) + protocolSuperset, Some(leaderInstanceId), sessionTimeout, rebalanceTimeout) val followerResponseFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, - protocolSuperset, Some(followerInstanceId), sessionTimeout) + protocolSuperset, Some(followerInstanceId), sessionTimeout, rebalanceTimeout) // The goal for two timer advance is to let first group initial join complete and set newMemberAdded flag to false. Next advance is // to trigger the rebalance as needed for follower delayed join. One large time advance won't help because we could only populate one // delayed join from purgatory and the new delayed op is created at that time and never be triggered. @@ -2941,6 +2987,29 @@ class GroupCoordinatorTest { assertEquals(Errors.ILLEGAL_GENERATION, commitOffsetResult(tp)) } + @Test + def testManualCommitOffsetShouldNotValidateMemberIdAndInstanceId(): Unit = { + val tp = new TopicPartition("topic", 0) + + var commitOffsetResult = commitOffsets( + groupId, + JoinGroupRequest.UNKNOWN_MEMBER_ID, + -1, + Map(tp -> offsetAndMetadata(0)), + Some("instance-id") + ) + assertEquals(Errors.NONE, commitOffsetResult(tp)) + + commitOffsetResult = commitOffsets( + groupId, + "unknown", + -1, + Map(tp -> offsetAndMetadata(0)), + None + ) + assertEquals(Errors.NONE, commitOffsetResult(tp)) + } + @Test def testTxnCommitOffsetWithFencedInstanceId(): Unit = { val tp = new TopicPartition("topic", 0) diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala index 688d6e83b0dc9..dba089c974ceb 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala @@ -2134,6 +2134,8 @@ class GroupMetadataManagerTest { group.updateMember( member, List(("protocol", ConsumerProtocol.serializeSubscription(subscriptionTopic1).array())), + member.rebalanceTimeoutMs, + member.sessionTimeoutMs, null ) diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala index eb0748d537c2d..e750024c9fb4d 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala @@ -22,7 +22,7 @@ import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.utils.{Time, MockTime} +import org.apache.kafka.common.utils.{MockTime, Time} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, Test} @@ -557,6 +557,21 @@ class GroupMetadataTest { assertFalse(group.hasPendingOffsetCommitsFromProducer(producerId)) } + @Test + def testUpdateMember(): Unit = { + val member = new MemberMetadata(memberId, None, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, + protocolType, List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte]))) + group.add(member) + + val newRebalanceTimeout = 120000 + val newSessionTimeout = 20000 + group.updateMember(member, List(("roundrobin", Array[Byte]())), newRebalanceTimeout, newSessionTimeout, null) + + assertEquals(group.rebalanceTimeoutMs, newRebalanceTimeout) + assertEquals(member.sessionTimeoutMs, newSessionTimeout) + } + + @Test def testReplaceGroupInstanceWithNonExistingMember(): Unit = { val newMemberId = "newMemberId" @@ -605,6 +620,30 @@ class GroupMetadataTest { assertFalse(member.isAwaitingJoin) } + @Test + def testInvokeJoinCallbackFails(): Unit = { + val member = new MemberMetadata(memberId, None, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, + protocolType, List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte]))) + + var shouldFail = true + var result: Option[JoinGroupResult] = None + def joinCallback(joinGroupResult: JoinGroupResult): Unit = { + if (shouldFail) { + shouldFail = false + throw new Exception("Something went wrong!") + } else { + result = Some(joinGroupResult) + } + } + + group.add(member, joinCallback) + + group.maybeInvokeJoinCallback(member, JoinGroupResult(member.memberId, Errors.NONE)) + + assertEquals(Errors.UNKNOWN_SERVER_ERROR, result.get.error) + assertFalse(member.isAwaitingJoin) + } + @Test def testNotInvokeJoinCallback(): Unit = { val member = new MemberMetadata(memberId, None, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, @@ -616,6 +655,31 @@ class GroupMetadataTest { assertFalse(member.isAwaitingJoin) } + @Test + def testInvokeSyncCallbackFails(): Unit = { + val member = new MemberMetadata(memberId, None, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, + protocolType, List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte]))) + + var shouldFail = true + var result: Option[SyncGroupResult] = None + def syncCallback(syncGroupResult: SyncGroupResult): Unit = { + if (shouldFail) { + shouldFail = false + throw new Exception("Something went wrong!") + } else { + result = Some(syncGroupResult) + } + } + + group.add(member) + member.awaitingSyncCallback = syncCallback + + val invoked = group.maybeInvokeSyncCallback(member, SyncGroupResult(Errors.NONE)) + assertTrue(invoked) + assertEquals(Errors.UNKNOWN_SERVER_ERROR, result.get.error) + assertFalse(member.isAwaitingSync) + } + @Test def testInvokeSyncCallback(): Unit = { val member = new MemberMetadata(memberId, None, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala index eefe61d17d690..666a3c363ffcf 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala @@ -19,6 +19,7 @@ package kafka.coordinator.transaction import kafka.server.BrokerToControllerChannelManager import kafka.zk.{KafkaZkClient, ProducerIdBlockZNode} import org.apache.kafka.common.KafkaException +import org.apache.kafka.common.errors.CoordinatorLoadInProgressException import org.apache.kafka.common.message.AllocateProducerIdsResponseData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.AllocateProducerIdsResponse @@ -30,7 +31,6 @@ import org.junit.jupiter.params.provider.{EnumSource, ValueSource} import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, anyString} import org.mockito.Mockito.{mock, when} - import java.util.stream.IntStream class ProducerIdManagerTest { @@ -39,10 +39,13 @@ class ProducerIdManagerTest { val zkClient: KafkaZkClient = mock(classOf[KafkaZkClient]) // Mutable test implementation that lets us easily set the idStart and error - class MockProducerIdManager(val brokerId: Int, var idStart: Long, val idLen: Int, var error: Errors = Errors.NONE) + class MockProducerIdManager(val brokerId: Int, var idStart: Long, val idLen: Int, var error: Errors = Errors.NONE, timeout: Boolean = false) extends RPCProducerIdManager(brokerId, () => 1, brokerToController, 100) { override private[transaction] def sendRequest(): Unit = { + if (timeout) + return + if (error == Errors.NONE) { handleAllocateProducerIdsResponse(new AllocateProducerIdsResponse( new AllocateProducerIdsResponseData().setProducerIdStart(idStart).setProducerIdLen(idLen))) @@ -93,6 +96,12 @@ class ProducerIdManagerTest { assertEquals(pid2 + ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE * 2, manager2.generateProducerId()) } + @Test + def testRPCProducerIdManagerThrowsConcurrentTransactions(): Unit = { + val manager1 = new MockProducerIdManager(0, 0, 0, timeout = true) + assertThrows(classOf[CoordinatorLoadInProgressException], () => manager1.generateProducerId()) + } + @Test def testExceedProducerIdLimitZk(): Unit = { when(zkClient.getDataAndVersion(anyString)).thenAnswer(_ => { diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 43226507808cd..fe1922cc2b8f6 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -232,7 +232,7 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness { TestUtils.deleteTopicWithAdmin( admin = admin, topic = topic, - brokers = brokers) + brokers = aliveBrokers) } } else { adminZkClient.deleteTopic(topic) diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 0d41a5073bf02..c6379ff3f3341 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -38,6 +38,7 @@ import org.mockito.ArgumentMatchers import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{mock, reset, times, verify, when} +import java.util.concurrent.ConcurrentMap import scala.annotation.nowarn import scala.collection.mutable.ListBuffer import scala.collection.{Iterable, Map, mutable} @@ -117,7 +118,7 @@ class LogLoaderTest { override def loadLog(logDir: File, hadCleanShutdown: Boolean, recoveryPoints: Map[TopicPartition, Long], logStartOffsets: Map[TopicPartition, Long], defaultConfig: LogConfig, - topicConfigs: Map[String, LogConfig]): UnifiedLog = { + topicConfigs: Map[String, LogConfig], numRemainingSegments: ConcurrentMap[String, Int]): UnifiedLog = { if (simulateError.hasError) { simulateError.errorType match { case ErrorTypes.KafkaStorageExceptionWithIOExceptionCause => diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 5353df6db3855..1b2dd7809f3fb 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -17,10 +17,10 @@ package kafka.log -import com.yammer.metrics.core.MetricName +import com.yammer.metrics.core.{Gauge, MetricName} import kafka.server.checkpoints.OffsetCheckpointFile import kafka.server.metadata.{ConfigRepository, MockConfigRepository} -import kafka.server.{FetchDataInfo, FetchLogEnd} +import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchLogEnd, LogDirFailureChannel} import kafka.utils._ import org.apache.directory.api.util.FileUtils import org.apache.kafka.common.errors.OffsetOutOfRangeException @@ -29,16 +29,17 @@ import org.apache.kafka.common.{KafkaException, TopicPartition} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any -import org.mockito.{ArgumentMatchers, Mockito} -import org.mockito.Mockito.{doAnswer, mock, never, spy, times, verify} +import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito} +import org.mockito.Mockito.{doAnswer, doNothing, mock, never, spy, times, verify} + import java.io._ import java.nio.file.Files -import java.util.concurrent.Future +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, Future} import java.util.{Collections, Properties} - import org.apache.kafka.server.metrics.KafkaYammerMetrics -import scala.collection.mutable +import scala.collection.{Map, mutable} +import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ import scala.util.{Failure, Try} @@ -421,12 +422,14 @@ class LogManagerTest { } private def createLogManager(logDirs: Seq[File] = Seq(this.logDir), - configRepository: ConfigRepository = new MockConfigRepository): LogManager = { + configRepository: ConfigRepository = new MockConfigRepository, + recoveryThreadsPerDataDir: Int = 1): LogManager = { TestUtils.createLogManager( defaultConfig = logConfig, configRepository = configRepository, logDirs = logDirs, - time = this.time) + time = this.time, + recoveryThreadsPerDataDir = recoveryThreadsPerDataDir) } @Test @@ -638,6 +641,205 @@ class LogManagerTest { assertTrue(logManager.partitionsInitializing.isEmpty) } + private def appendRecordsToLog(time: MockTime, parentLogDir: File, partitionId: Int, brokerTopicStats: BrokerTopicStats, expectedSegmentsPerLog: Int): Unit = { + def createRecord = TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds) + val tpFile = new File(parentLogDir, s"$name-$partitionId") + val segmentBytes = 1024 + + val log = LogTestUtils.createLog(tpFile, logConfig, brokerTopicStats, time.scheduler, time, 0, 0, + 5 * 60 * 1000, 60 * 60 * 1000, LogManager.ProducerIdExpirationCheckIntervalMs) + + assertTrue(expectedSegmentsPerLog > 0) + // calculate numMessages to append to logs. It'll create "expectedSegmentsPerLog" log segments with segment.bytes=1024 + val numMessages = Math.floor(segmentBytes * expectedSegmentsPerLog / createRecord.sizeInBytes).asInstanceOf[Int] + try { + for (_ <- 0 until numMessages) { + log.appendAsLeader(createRecord, leaderEpoch = 0) + } + + assertEquals(expectedSegmentsPerLog, log.numberOfSegments) + } finally { + log.close() + } + } + + private def verifyRemainingLogsToRecoverMetric(spyLogManager: LogManager, expectedParams: Map[String, Int]): Unit = { + val spyLogManagerClassName = spyLogManager.getClass().getSimpleName + // get all `remainingLogsToRecover` metrics + val logMetrics: ArrayBuffer[Gauge[Int]] = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala + .filter { case (metric, _) => metric.getType == s"$spyLogManagerClassName" && metric.getName == "remainingLogsToRecover" } + .map { case (_, gauge) => gauge } + .asInstanceOf[ArrayBuffer[Gauge[Int]]] + + assertEquals(expectedParams.size, logMetrics.size) + + val capturedPath: ArgumentCaptor[String] = ArgumentCaptor.forClass(classOf[String]) + + val expectedCallTimes = expectedParams.values.sum + verify(spyLogManager, times(expectedCallTimes)).decNumRemainingLogs(any[ConcurrentMap[String, Int]], capturedPath.capture()); + + val paths = capturedPath.getAllValues + expectedParams.foreach { + case (path, totalLogs) => + // make sure each path is called "totalLogs" times, which means it is decremented to 0 in the end + assertEquals(totalLogs, Collections.frequency(paths, path)) + } + + // expected the end value is 0 + logMetrics.foreach { gauge => assertEquals(0, gauge.value()) } + } + + private def verifyRemainingSegmentsToRecoverMetric(spyLogManager: LogManager, + logDirs: Seq[File], + recoveryThreadsPerDataDir: Int, + mockMap: ConcurrentHashMap[String, Int], + expectedParams: Map[String, Int]): Unit = { + val spyLogManagerClassName = spyLogManager.getClass().getSimpleName + // get all `remainingSegmentsToRecover` metrics + val logSegmentMetrics: ArrayBuffer[Gauge[Int]] = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala + .filter { case (metric, _) => metric.getType == s"$spyLogManagerClassName" && metric.getName == "remainingSegmentsToRecover" } + .map { case (_, gauge) => gauge } + .asInstanceOf[ArrayBuffer[Gauge[Int]]] + + // expected each log dir has 1 metrics for each thread + assertEquals(recoveryThreadsPerDataDir * logDirs.size, logSegmentMetrics.size) + + val capturedThreadName: ArgumentCaptor[String] = ArgumentCaptor.forClass(classOf[String]) + val capturedNumRemainingSegments: ArgumentCaptor[Int] = ArgumentCaptor.forClass(classOf[Int]) + + // Since we'll update numRemainingSegments from totalSegments to 0 for each thread, so we need to add 1 here + val expectedCallTimes = expectedParams.values.map( num => num + 1 ).sum + verify(mockMap, times(expectedCallTimes)).put(capturedThreadName.capture(), capturedNumRemainingSegments.capture()); + + // expected the end value is 0 + logSegmentMetrics.foreach { gauge => assertEquals(0, gauge.value()) } + + val threadNames = capturedThreadName.getAllValues + val numRemainingSegments = capturedNumRemainingSegments.getAllValues + + expectedParams.foreach { + case (threadName, totalSegments) => + // make sure we update the numRemainingSegments from totalSegments to 0 in order for each thread + var expectedCurRemainingSegments = totalSegments + 1 + for (i <- 0 until threadNames.size) { + if (threadNames.get(i).contains(threadName)) { + expectedCurRemainingSegments -= 1 + assertEquals(expectedCurRemainingSegments, numRemainingSegments.get(i)) + } + } + assertEquals(0, expectedCurRemainingSegments) + } + } + + private def verifyLogRecoverMetricsRemoved(spyLogManager: LogManager): Unit = { + val spyLogManagerClassName = spyLogManager.getClass().getSimpleName + // get all `remainingLogsToRecover` metrics + def logMetrics: mutable.Set[MetricName] = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala + .filter { metric => metric.getType == s"$spyLogManagerClassName" && metric.getName == "remainingLogsToRecover" } + + assertTrue(logMetrics.isEmpty) + + // get all `remainingSegmentsToRecover` metrics + val logSegmentMetrics: mutable.Set[MetricName] = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala + .filter { metric => metric.getType == s"$spyLogManagerClassName" && metric.getName == "remainingSegmentsToRecover" } + + assertTrue(logSegmentMetrics.isEmpty) + } + + @Test + def testLogRecoveryMetrics(): Unit = { + logManager.shutdown() + val logDir1 = TestUtils.tempDir() + val logDir2 = TestUtils.tempDir() + val logDirs = Seq(logDir1, logDir2) + val recoveryThreadsPerDataDir = 2 + // create logManager with expected recovery thread number + logManager = createLogManager(logDirs, recoveryThreadsPerDataDir = recoveryThreadsPerDataDir) + val spyLogManager = spy(logManager) + + assertEquals(2, spyLogManager.liveLogDirs.size) + + val mockTime = new MockTime() + val mockMap = mock(classOf[ConcurrentHashMap[String, Int]]) + val mockBrokerTopicStats = mock(classOf[BrokerTopicStats]) + val expectedSegmentsPerLog = 2 + + // create log segments for log recovery in each log dir + appendRecordsToLog(mockTime, logDir1, 0, mockBrokerTopicStats, expectedSegmentsPerLog) + appendRecordsToLog(mockTime, logDir2, 1, mockBrokerTopicStats, expectedSegmentsPerLog) + + // intercept loadLog method to pass expected parameter to do log recovery + doAnswer { invocation => + val dir: File = invocation.getArgument(0) + val topicConfigOverrides: mutable.Map[String, LogConfig] = invocation.getArgument(5) + + val topicPartition = UnifiedLog.parseTopicPartitionName(dir) + val config = topicConfigOverrides.getOrElse(topicPartition.topic, logConfig) + + UnifiedLog( + dir = dir, + config = config, + logStartOffset = 0, + recoveryPoint = 0, + maxTransactionTimeoutMs = 5 * 60 * 1000, + maxProducerIdExpirationMs = 5 * 60 * 1000, + producerIdExpirationCheckIntervalMs = LogManager.ProducerIdExpirationCheckIntervalMs, + scheduler = mockTime.scheduler, + time = mockTime, + brokerTopicStats = mockBrokerTopicStats, + logDirFailureChannel = mock(classOf[LogDirFailureChannel]), + // not clean shutdown + lastShutdownClean = false, + topicId = None, + keepPartitionMetadataFile = false, + // pass mock map for verification later + numRemainingSegments = mockMap) + + }.when(spyLogManager).loadLog(any[File], any[Boolean], any[Map[TopicPartition, Long]], any[Map[TopicPartition, Long]], + any[LogConfig], any[Map[String, LogConfig]], any[ConcurrentMap[String, Int]]) + + // do nothing for removeLogRecoveryMetrics for metrics verification + doNothing().when(spyLogManager).removeLogRecoveryMetrics() + + // start the logManager to do log recovery + spyLogManager.startup(Set.empty) + + // make sure log recovery metrics are added and removed + verify(spyLogManager, times(1)).addLogRecoveryMetrics(any[ConcurrentMap[String, Int]], any[ConcurrentMap[String, Int]]) + verify(spyLogManager, times(1)).removeLogRecoveryMetrics() + + // expected 1 log in each log dir since we created 2 partitions with 2 log dirs + val expectedRemainingLogsParams = Map[String, Int](logDir1.getAbsolutePath -> 1, logDir2.getAbsolutePath -> 1) + verifyRemainingLogsToRecoverMetric(spyLogManager, expectedRemainingLogsParams) + + val expectedRemainingSegmentsParams = Map[String, Int]( + logDir1.getAbsolutePath -> expectedSegmentsPerLog, logDir2.getAbsolutePath -> expectedSegmentsPerLog) + verifyRemainingSegmentsToRecoverMetric(spyLogManager, logDirs, recoveryThreadsPerDataDir, mockMap, expectedRemainingSegmentsParams) + } + + @Test + def testLogRecoveryMetricsShouldBeRemovedAfterLogRecovered(): Unit = { + logManager.shutdown() + val logDir1 = TestUtils.tempDir() + val logDir2 = TestUtils.tempDir() + val logDirs = Seq(logDir1, logDir2) + val recoveryThreadsPerDataDir = 2 + // create logManager with expected recovery thread number + logManager = createLogManager(logDirs, recoveryThreadsPerDataDir = recoveryThreadsPerDataDir) + val spyLogManager = spy(logManager) + + assertEquals(2, spyLogManager.liveLogDirs.size) + + // start the logManager to do log recovery + spyLogManager.startup(Set.empty) + + // make sure log recovery metrics are added and removed once + verify(spyLogManager, times(1)).addLogRecoveryMetrics(any[ConcurrentMap[String, Int]], any[ConcurrentMap[String, Int]]) + verify(spyLogManager, times(1)).removeLogRecoveryMetrics() + + verifyLogRecoverMetricsRemoved(spyLogManager) + } + @Test def testMetricsExistWhenLogIsRecreatedBeforeDeletion(): Unit = { val topicName = "metric-test" diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index f6b58d78ce8cc..50af76f556ca2 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -28,6 +28,7 @@ import org.apache.kafka.common.utils.{Time, Utils} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse} import java.nio.file.Files +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import scala.collection.Iterable import scala.jdk.CollectionConverters._ @@ -83,7 +84,8 @@ object LogTestUtils { producerIdExpirationCheckIntervalMs: Int = LogManager.ProducerIdExpirationCheckIntervalMs, lastShutdownClean: Boolean = true, topicId: Option[Uuid] = None, - keepPartitionMetadataFile: Boolean = true): UnifiedLog = { + keepPartitionMetadataFile: Boolean = true, + numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int]): UnifiedLog = { UnifiedLog( dir = dir, config = config, @@ -98,7 +100,8 @@ object LogTestUtils { logDirFailureChannel = new LogDirFailureChannel(10), lastShutdownClean = lastShutdownClean, topicId = topicId, - keepPartitionMetadataFile = keepPartitionMetadataFile + keepPartitionMetadataFile = keepPartitionMetadataFile, + numRemainingSegments = numRemainingSegments ) } diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 57409a1f0384e..560e84c3a586e 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -3454,6 +3454,26 @@ class UnifiedLogTest { assertFalse(newDir.exists()) } + @Test + def testMaybeUpdateHighWatermarkAsFollower(): Unit = { + val logConfig = LogTestUtils.createLogConfig() + val log = createLog(logDir, logConfig) + + for (i <- 0 until 100) { + val records = TestUtils.singletonRecords(value = s"test$i".getBytes) + log.appendAsLeader(records, leaderEpoch = 0) + } + + assertEquals(Some(99L), log.maybeUpdateHighWatermark(99L)) + assertEquals(None, log.maybeUpdateHighWatermark(99L)) + + assertEquals(Some(100L), log.maybeUpdateHighWatermark(100L)) + assertEquals(None, log.maybeUpdateHighWatermark(100L)) + + // bound by the log end offset + assertEquals(None, log.maybeUpdateHighWatermark(101L)) + } + private def appendTransactionalToBuffer(buffer: ByteBuffer, producerId: Long, producerEpoch: Short, diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index b21fe877f2082..29de3c0f24288 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -233,16 +233,21 @@ class MetricsTest extends KafkaServerTestHarness with Logging { @ValueSource(strings = Array("kraft")) def testKRaftControllerMetrics(quorum: String): Unit = { val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics - - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=ActiveControllerCount"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=OfflinePartitionsCount"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=PreferredReplicaImbalanceCount"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=GlobalTopicCount"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=GlobalPartitionCount"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=LastCommittedRecordOffset"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=LastAppliedRecordOffset"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=LastAppliedRecordTimestamp"), 1) - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.controller:type=KafkaController,name=LastAppliedRecordLagMs"), 1) + Set( + "kafka.controller:type=KafkaController,name=ActiveControllerCount", + "kafka.controller:type=KafkaController,name=GlobalPartitionCount", + "kafka.controller:type=KafkaController,name=GlobalTopicCount", + "kafka.controller:type=KafkaController,name=LastAppliedRecordLagMs", + "kafka.controller:type=KafkaController,name=LastAppliedRecordOffset", + "kafka.controller:type=KafkaController,name=LastAppliedRecordTimestamp", + "kafka.controller:type=KafkaController,name=LastCommittedRecordOffset", + "kafka.controller:type=KafkaController,name=MetadataErrorCount", + "kafka.controller:type=KafkaController,name=OfflinePartitionsCount", + "kafka.controller:type=KafkaController,name=PreferredReplicaImbalanceCount", + ).foreach(expected => { + assertEquals(1, metrics.keySet.asScala.count(_.getMBeanName.equals(expected)), + s"Unable to find ${expected}") + }) } /** diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala index f8fac503d6ec1..9d7a93db94c74 100644 --- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala +++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala @@ -82,23 +82,23 @@ class RaftManagerTest { } @Test - def testSentinelNodeIdIfBrokerRoleOnly(): Unit = { + def testNodeIdPresentIfBrokerRoleOnly(): Unit = { val raftManager = instantiateRaftManagerWithConfigs(new TopicPartition("__raft_id_test", 0), "broker", "1") - assertFalse(raftManager.client.nodeId.isPresent) + assertEquals(1, raftManager.client.nodeId.getAsInt) raftManager.shutdown() } @Test def testNodeIdPresentIfControllerRoleOnly(): Unit = { val raftManager = instantiateRaftManagerWithConfigs(new TopicPartition("__raft_id_test", 0), "controller", "1") - assertTrue(raftManager.client.nodeId.getAsInt == 1) + assertEquals(1, raftManager.client.nodeId.getAsInt) raftManager.shutdown() } @Test def testNodeIdPresentIfColocated(): Unit = { val raftManager = instantiateRaftManagerWithConfigs(new TopicPartition("__raft_id_test", 0), "controller,broker", "1") - assertTrue(raftManager.client.nodeId.getAsInt == 1) + assertEquals(1, raftManager.client.nodeId.getAsInt) raftManager.shutdown() } diff --git a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala index ecee2cd19c4e5..627a939ddcf77 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala @@ -163,9 +163,9 @@ abstract class AbstractCreateTopicsRequestTest extends BaseRequestTest { if (actual == null) { throw new RuntimeException(s"No response data found for topic $topicName") } - assertEquals(expected.error.code(), actual.errorCode(), "The response error should match") + assertEquals(expected.error.code(), actual.errorCode(), "The response error code should match") if (checkErrorMessage) { - assertEquals(expected.message, actual.errorMessage()) + assertEquals(expected.message, actual.errorMessage(), "The response error message should match") } // If no error validate topic exists if (expectedError.isSuccess && !request.data.validateOnly) { diff --git a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala index 0a98d2626cd23..74320e62b49a1 100644 --- a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala @@ -17,13 +17,16 @@ package kafka.server -import java.util.Properties +import kafka.utils.TestInfoUtils +import java.util.Properties import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AddPartitionsToTxnRequest, AddPartitionsToTxnResponse} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{BeforeEach, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.jdk.CollectionConverters._ @@ -37,11 +40,12 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { super.setUp(testInfo) - createTopic(topic1, numPartitions, servers.size, new Properties()) + createTopic(topic1, numPartitions, brokers.size, new Properties()) } - @Test - def shouldReceiveOperationNotAttemptedWhenOtherPartitionHasError(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def shouldReceiveOperationNotAttemptedWhenOtherPartitionHasError(quorum: String): Unit = { // The basic idea is that we have one unknown topic and one created topic. We should get the 'UNKNOWN_TOPIC_OR_PARTITION' // error for the unknown topic and the 'OPERATION_NOT_ATTEMPTED' error for the known and authorized topic. val nonExistentTopic = new TopicPartition("unknownTopic", 0) @@ -58,7 +62,7 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest { List(createdTopicPartition, nonExistentTopic).asJava) .build() - val leaderId = servers.head.config.brokerId + val leaderId = brokers.head.config.brokerId val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(leaderId)) assertEquals(2, response.errors.size) diff --git a/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala new file mode 100644 index 0000000000000..5cb59573d1adf --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package unit.kafka.server + +import kafka.network.SocketServer +import kafka.server.{BrokerServer, ControllerServer, IntegrationTestUtils} +import kafka.test.ClusterInstance +import kafka.test.annotation.{ClusterTest, ClusterTestDefaults, Type} +import kafka.test.junit.ClusterTestExtensions +import kafka.test.junit.RaftClusterInvocationContext.RaftClusterInstance +import org.apache.kafka.common.message.AllocateProducerIdsRequestData +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests._ +import org.apache.kafka.server.common.ProducerIdsBlock +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.extension.ExtendWith +import org.junit.jupiter.api.{Tag, Timeout} + +@Timeout(120) +@ExtendWith(value = Array(classOf[ClusterTestExtensions])) +@ClusterTestDefaults(clusterType = Type.KRAFT) +@Tag("integration") +class AllocateProducerIdsRequestTest(cluster: ClusterInstance) { + + @ClusterTest + def testAllocateProducersIdSentToController(): Unit = { + val raftCluster = cluster.asInstanceOf[RaftClusterInstance] + val sourceBroker = raftCluster.brokers.findFirst().get() + + val controllerId = sourceBroker.raftManager.leaderAndEpoch.leaderId().getAsInt + val controllerServer = raftCluster.controllers() + .filter(_.config.nodeId == controllerId) + .findFirst() + .get() + + val allocateResponse = sendAndReceiveAllocateProducerIds(sourceBroker, controllerServer) + assertEquals(Errors.NONE, allocateResponse.error) + assertEquals(ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE, allocateResponse.data.producerIdLen) + assertTrue(allocateResponse.data.producerIdStart >= 0) + } + + @ClusterTest(controllers = 3) + def testAllocateProducersIdSentToNonController(): Unit = { + val raftCluster = cluster.asInstanceOf[RaftClusterInstance] + val sourceBroker = raftCluster.brokers.findFirst().get() + + val controllerId = sourceBroker.raftManager.leaderAndEpoch.leaderId().getAsInt + val controllerServer = raftCluster.controllers() + .filter(_.config.nodeId != controllerId) + .findFirst() + .get() + + val allocateResponse = sendAndReceiveAllocateProducerIds(sourceBroker, controllerServer) + assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(allocateResponse.data.errorCode)) + } + + private def sendAndReceiveAllocateProducerIds( + sourceBroker: BrokerServer, + controllerServer: ControllerServer + ): AllocateProducerIdsResponse = { + val allocateRequest = new AllocateProducerIdsRequest.Builder( + new AllocateProducerIdsRequestData() + .setBrokerId(sourceBroker.config.brokerId) + .setBrokerEpoch(sourceBroker.lifecycleManager.brokerEpoch) + ).build() + + connectAndReceive( + controllerServer.socketServer, + allocateRequest + ) + } + + private def connectAndReceive( + controllerSocketServer: SocketServer, + request: AllocateProducerIdsRequest + ): AllocateProducerIdsResponse = { + IntegrationTestUtils.connectAndReceive[AllocateProducerIdsResponse]( + request, + controllerSocketServer, + cluster.controllerListenerName.get + ) + } + +} diff --git a/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala index 4670dc4fd5a3a..458973700bdf7 100644 --- a/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala @@ -46,11 +46,20 @@ class BaseFetchRequestTest extends BaseRequestTest { super.tearDown() } - protected def createFetchRequest(maxResponseBytes: Int, maxPartitionBytes: Int, topicPartitions: Seq[TopicPartition], - offsetMap: Map[TopicPartition, Long], - version: Short): FetchRequest = { - FetchRequest.Builder.forConsumer(version, Int.MaxValue, 0, createPartitionMap(maxPartitionBytes, topicPartitions, offsetMap)) - .setMaxBytes(maxResponseBytes).build() + protected def createConsumerFetchRequest( + maxResponseBytes: Int, + maxPartitionBytes: Int, + topicPartitions: Seq[TopicPartition], + offsetMap: Map[TopicPartition, Long], + version: Short, + maxWaitMs: Int = Int.MaxValue, + minBytes: Int = 0, + rackId: String = "" + ): FetchRequest = { + FetchRequest.Builder.forConsumer(version, maxWaitMs, minBytes, createPartitionMap(maxPartitionBytes, topicPartitions, offsetMap)) + .setMaxBytes(maxResponseBytes) + .rackId(rackId) + .build() } protected def createPartitionMap(maxPartitionBytes: Int, topicPartitions: Seq[TopicPartition], @@ -64,8 +73,8 @@ class BaseFetchRequestTest extends BaseRequestTest { partitionMap } - protected def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse = { - connectAndReceive[FetchResponse](request, destination = brokerSocketServer(leaderId)) + protected def sendFetchRequest(brokerId: Int, request: FetchRequest): FetchResponse = { + connectAndReceive[FetchResponse](request, destination = brokerSocketServer(brokerId)) } protected def initProducer(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 86a0c8547057b..969d57c0dd202 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -48,9 +48,9 @@ import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.{ElectionType, Uuid} import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT -import org.apache.kafka.controller.{Controller, ControllerRequestContext} +import org.apache.kafka.controller.{Controller, ControllerRequestContext, ResultOrError} import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer} -import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.common.{ApiMessageAndVersion, ProducerIdsBlock} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.junit.jupiter.params.ParameterizedTest @@ -61,8 +61,9 @@ import org.mockito.{ArgumentCaptor, ArgumentMatchers} import java.net.InetAddress import java.util -import java.util.Collections.singletonList +import java.util.Collections.{singleton, singletonList, singletonMap} import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit} +import java.util.concurrent.atomic.AtomicReference import java.util.{Collections, Properties} import scala.annotation.nowarn import scala.jdk.CollectionConverters._ @@ -553,7 +554,8 @@ class ControllerApisTest { setTopicId(new Uuid(0L, 2L)). setTopicConfigErrorCode(TOPIC_AUTHORIZATION_FAILED.code()), new CreatableTopicResult().setName("quux"). - setErrorCode(TOPIC_AUTHORIZATION_FAILED.code())) + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code()). + setErrorMessage("Authorization failed.")) assertEquals(expectedResponse, controllerApis.createTopics(ANONYMOUS_CONTEXT, request, false, _ => Set("baz", "indescribable"), @@ -875,6 +877,78 @@ class ControllerApisTest { assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(response.data.errorCode)) } + @Test + def testDeleteTopicsReturnsNotController(): Unit = { + val topicId = Uuid.randomUuid() + val topicName = "foo" + val controller = mock(classOf[Controller]) + val controllerApis = createControllerApis(None, controller) + + val findNamesFuture = CompletableFuture.completedFuture( + singletonMap(topicId, new ResultOrError(topicName)) + ) + when(controller.findTopicNames( + any[ControllerRequestContext], + ArgumentMatchers.eq(singleton(topicId)) + )).thenReturn(findNamesFuture) + + val findIdsFuture = CompletableFuture.completedFuture( + Collections.emptyMap[String, ResultOrError[Uuid]]() + ) + when(controller.findTopicIds( + any[ControllerRequestContext], + ArgumentMatchers.eq(Collections.emptySet()) + )).thenReturn(findIdsFuture) + + val deleteFuture = new CompletableFuture[util.Map[Uuid, ApiError]]() + deleteFuture.completeExceptionally(new NotControllerException("Controller has moved")) + when(controller.deleteTopics( + any[ControllerRequestContext], + ArgumentMatchers.eq(singleton(topicId)) + )).thenReturn(deleteFuture) + + val request = new DeleteTopicsRequest.Builder( + new DeleteTopicsRequestData().setTopics(singletonList( + new DeleteTopicState().setTopicId(topicId) + )) + ).build() + + val response = handleRequest[DeleteTopicsResponse](request, controllerApis) + val topicIdResponse = response.data.responses.asScala.find(_.topicId == topicId).get + assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(topicIdResponse.errorCode)) + } + + @Test + def testAllocateProducerIdsReturnsNotController(): Unit = { + val controller = mock(classOf[Controller]) + val controllerApis = createControllerApis(None, controller) + + // We construct the future here to mimic the logic in `QuorumController.allocateProducerIds`. + // When an exception is raised on the original future, the `thenApply` future is also completed + // exceptionally, but the underlying cause is wrapped in a `CompletionException`. + val future = new CompletableFuture[ProducerIdsBlock] + val thenApplyFuture = future.thenApply[AllocateProducerIdsResponseData] { result => + new AllocateProducerIdsResponseData() + .setProducerIdStart(result.firstProducerId()) + .setProducerIdLen(result.size()) + } + future.completeExceptionally(new NotControllerException("Controller has moved")) + + val request = new AllocateProducerIdsRequest.Builder( + new AllocateProducerIdsRequestData() + .setBrokerId(4) + .setBrokerEpoch(93234) + ).build() + + when(controller.allocateProducerIds( + any[ControllerRequestContext], + ArgumentMatchers.eq(request.data) + )).thenReturn(thenApplyFuture) + + val response = handleRequest[AllocateProducerIdsResponse](request, controllerApis) + assertEquals(Errors.NOT_CONTROLLER, response.error) + } + private def handleRequest[T <: AbstractResponse]( request: AbstractRequest, controllerApis: ControllerApis @@ -902,6 +976,35 @@ class ControllerApisTest { } } + @Test + def testCompletableFutureExceptions(): Unit = { + // This test simulates an error in a completable future as we return from the controller. We need to ensure + // that any exception throw in the completion phase is properly captured and translated to an error response. + val request = buildRequest(new FetchRequest(new FetchRequestData(), 12)) + val response = new FetchResponseData() + val responseFuture = new CompletableFuture[ApiMessage]() + val errorResponseFuture = new AtomicReference[AbstractResponse]() + when(raftManager.handleRequest(any(), any(), any())).thenReturn(responseFuture) + when(requestChannel.sendResponse(any(), any(), any())).thenAnswer { _ => + // Simulate an encoding failure in the initial fetch response + throw new UnsupportedVersionException("Something went wrong") + }.thenAnswer { invocation => + val resp = invocation.getArgument(1, classOf[AbstractResponse]) + errorResponseFuture.set(resp) + } + + // Calling handle does not block since we do not call get() in ControllerApis + createControllerApis(None, + new MockController.Builder().build()).handle(request, null) + + // When we complete this future, the completion stages will fire (including the error handler in ControllerApis#request) + responseFuture.complete(response) + + // Now we should get an error response with UNSUPPORTED_VERSION + val errorResponse = errorResponseFuture.get() + assertEquals(1, errorResponse.errorCounts().getOrDefault(Errors.UNSUPPORTED_VERSION, 0)) + } + @AfterEach def tearDown(): Unit = { quotas.shutdown() diff --git a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala index c89910ed23138..36a8d71fb9759 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala @@ -39,8 +39,8 @@ class ControllerConfigurationValidatorTest { @Test def testInvalidTopicNameRejected(): Unit = { - assertEquals("Topic name \"(<-invalid->)\" is illegal, it contains a character " + - "other than ASCII alphanumerics, '.', '_' and '-'", + assertEquals("Topic name is invalid: '(<-invalid->)' contains " + + "one or more characters other than ASCII alphanumerics, '.', '_' and '-'", assertThrows(classOf[InvalidTopicException], () => validator.validate( new ConfigResource(TOPIC, "(<-invalid->)"), emptyMap())). getMessage()) } diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala index 57834234cc101..a193db284c461 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala @@ -19,6 +19,7 @@ package kafka.server import kafka.utils._ import org.apache.kafka.common.Uuid +import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.CreateTopicsRequestData import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection import org.apache.kafka.common.protocol.ApiKeys @@ -27,7 +28,6 @@ import org.apache.kafka.common.requests.CreateTopicsRequest import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource - import scala.jdk.CollectionConverters._ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { @@ -197,4 +197,13 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { assertEquals(Uuid.ZERO_UUID, topicResponse.topicId()) } } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testCreateClusterMetadataTopic(quorum: String): Unit = { + validateErrorCreateTopicsRequests( + topicsReq(Seq(topicReq(Topic.CLUSTER_METADATA_TOPIC_NAME))), + Map(Topic.CLUSTER_METADATA_TOPIC_NAME -> error(Errors.TOPIC_AUTHORIZATION_FAILED, Some("Authorization failed."))) + ) + } } diff --git a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala index 644f21ff3f648..629f203169d42 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala @@ -32,10 +32,46 @@ import org.apache.kafka.common.requests.MetadataResponse import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource + +import scala.collection.Seq import scala.jdk.CollectionConverters._ class DeleteTopicsRequestTest extends BaseRequestTest with Logging { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testTopicDeletionClusterHasOfflinePartitions(quorum: String): Unit = { + // Create a two topics with one partition/replica. Make one of them offline. + val offlineTopic = "topic-1" + val onlineTopic = "topic-2" + createTopicWithAssignment(offlineTopic, Map[Int, Seq[Int]](0 -> Seq(0))) + createTopicWithAssignment(onlineTopic, Map[Int, Seq[Int]](0 -> Seq(1))) + killBroker(0) + ensureConsistentKRaftMetadata() + + // Ensure one topic partition is offline. + TestUtils.waitUntilTrue(() => { + aliveBrokers.head.metadataCache.getPartitionInfo(onlineTopic, 0).exists(_.leader() == 1) && + aliveBrokers.head.metadataCache.getPartitionInfo(offlineTopic, 0).exists(_.leader() == + MetadataResponse.NO_LEADER_ID) + }, "Topic partition is not offline") + + // Delete the newly created topic and topic with offline partition. See the deletion is + // successful. + deleteTopic(onlineTopic) + deleteTopic(offlineTopic) + ensureConsistentKRaftMetadata() + + // Restart the dead broker. + restartDeadBrokers() + + // Make sure the brokers no longer see any deleted topics. + TestUtils.waitUntilTrue(() => + !aliveBrokers.forall(_.metadataCache.contains(onlineTopic)) && + !aliveBrokers.forall(_.metadataCache.contains(offlineTopic)), + "The topics are found in the Broker's cache") + } + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @ValueSource(strings = Array("zk", "kraft")) def testValidDeleteTopicRequests(quorum: String): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala index f8da00f10e8f5..28a6f801235c8 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala @@ -74,20 +74,28 @@ class DescribeQuorumRequestTest(cluster: ClusterInstance) { val leaderId = partitionData.leaderId assertTrue(leaderId > 0) + assertTrue(partitionData.leaderEpoch() > 0) + assertTrue(partitionData.highWatermark() > 0) val leaderState = partitionData.currentVoters.asScala.find(_.replicaId == leaderId) .getOrElse(throw new AssertionError("Failed to find leader among current voter states")) assertTrue(leaderState.logEndOffset > 0) val voterData = partitionData.currentVoters.asScala + assertEquals(cluster.controllerIds().asScala, voterData.map(_.replicaId).toSet); + val observerData = partitionData.observers.asScala - assertEquals(1, voterData.size) - assertEquals(0, observerData.size) - voterData.foreach { state => - assertTrue(0 < state.replicaId) + assertEquals(cluster.brokerIds().asScala, observerData.map(_.replicaId).toSet); + + (voterData ++ observerData).foreach { state => assertTrue(0 < state.logEndOffset) - assertEquals(-1, state.lastFetchTimestamp) - assertEquals(-1, state.lastCaughtUpTimestamp) + if (version == 0) { + assertEquals(-1, state.lastFetchTimestamp) + assertEquals(-1, state.lastCaughtUpTimestamp) + } else { + assertNotEquals(-1, state.lastFetchTimestamp) + assertNotEquals(-1, state.lastCaughtUpTimestamp) + } } } } diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index 1a383a8fbcdf9..1bbde3ffb6b88 100755 --- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala @@ -35,19 +35,20 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.ByteUtils import org.apache.kafka.common.{TopicPartition, requests} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.jdk.CollectionConverters._ class EdgeCaseRequestTest extends KafkaServerTestHarness { def generateConfigs = { - val props = TestUtils.createBrokerConfig(1, zkConnect) + val props = TestUtils.createBrokerConfig(1, zkConnectOrNull) props.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, "false") List(KafkaConfig.fromProps(props)) } - private def socketServer = servers.head.socketServer + private def socketServer = brokers.head.socketServer private def connect(s: SocketServer = socketServer, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): Socket = { new Socket("localhost", s.boundPort(ListenerName.forSecurityProtocol(protocol))) @@ -116,8 +117,9 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { } } - @Test - def testProduceRequestWithNullClientId(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testProduceRequestWithNullClientId(quorum: String): Unit = { val topic = "topic" val topicPartition = new TopicPartition(topic, 0) val correlationId = -1 @@ -161,23 +163,27 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { assertEquals(Errors.NONE, Errors.forCode(partitionProduceResponse.errorCode), "There should be no error") } - @Test - def testHeaderOnlyRequest(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testHeaderOnlyRequest(quorum: String): Unit = { verifyDisconnect(requestHeaderBytes(ApiKeys.PRODUCE.id, 1)) } - @Test - def testInvalidApiKeyRequest(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testInvalidApiKeyRequest(quorum: String): Unit = { verifyDisconnect(requestHeaderBytes(-1, 0)) } - @Test - def testInvalidApiVersionRequest(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testInvalidApiVersionRequest(quorum: String): Unit = { verifyDisconnect(requestHeaderBytes(ApiKeys.PRODUCE.id, -1)) } - @Test - def testMalformedHeaderRequest(): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testMalformedHeaderRequest(quorum: String): Unit = { val serializedBytes = { // Only send apiKey and apiVersion val buffer = ByteBuffer.allocate( diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala index 401b07d29f2cd..27d6e7a68bc89 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala @@ -49,9 +49,9 @@ class FetchRequestTest extends BaseFetchRequestTest { val maxResponseBytes = 800 val maxPartitionBytes = 190 - def createFetchRequest(topicPartitions: Seq[TopicPartition], offsetMap: Map[TopicPartition, Long] = Map.empty, + def createConsumerFetchRequest(topicPartitions: Seq[TopicPartition], offsetMap: Map[TopicPartition, Long] = Map.empty, version: Short = ApiKeys.FETCH.latestVersion()): FetchRequest = - this.createFetchRequest(maxResponseBytes, maxPartitionBytes, topicPartitions, offsetMap, version) + this.createConsumerFetchRequest(maxResponseBytes, maxPartitionBytes, topicPartitions, offsetMap, version) val topicPartitionToLeader = createTopics(numTopics = 5, numPartitions = 6) val random = new Random(0) @@ -77,28 +77,28 @@ class FetchRequestTest extends BaseFetchRequestTest { // 1. Partitions with large messages at the end val shuffledTopicPartitions1 = random.shuffle(partitionsWithoutLargeMessages) ++ partitionsWithLargeMessages - val fetchRequest1 = createFetchRequest(shuffledTopicPartitions1) + val fetchRequest1 = createConsumerFetchRequest(shuffledTopicPartitions1) val fetchResponse1 = sendFetchRequest(leaderId, fetchRequest1) checkFetchResponse(shuffledTopicPartitions1, fetchResponse1, maxPartitionBytes, maxResponseBytes, messagesPerPartition) - val fetchRequest1V12 = createFetchRequest(shuffledTopicPartitions1, version = 12) + val fetchRequest1V12 = createConsumerFetchRequest(shuffledTopicPartitions1, version = 12) val fetchResponse1V12 = sendFetchRequest(leaderId, fetchRequest1V12) checkFetchResponse(shuffledTopicPartitions1, fetchResponse1V12, maxPartitionBytes, maxResponseBytes, messagesPerPartition, 12) // 2. Same as 1, but shuffled again val shuffledTopicPartitions2 = random.shuffle(partitionsWithoutLargeMessages) ++ partitionsWithLargeMessages - val fetchRequest2 = createFetchRequest(shuffledTopicPartitions2) + val fetchRequest2 = createConsumerFetchRequest(shuffledTopicPartitions2) val fetchResponse2 = sendFetchRequest(leaderId, fetchRequest2) checkFetchResponse(shuffledTopicPartitions2, fetchResponse2, maxPartitionBytes, maxResponseBytes, messagesPerPartition) - val fetchRequest2V12 = createFetchRequest(shuffledTopicPartitions2, version = 12) + val fetchRequest2V12 = createConsumerFetchRequest(shuffledTopicPartitions2, version = 12) val fetchResponse2V12 = sendFetchRequest(leaderId, fetchRequest2V12) checkFetchResponse(shuffledTopicPartitions2, fetchResponse2V12, maxPartitionBytes, maxResponseBytes, messagesPerPartition, 12) // 3. Partition with message larger than the partition limit at the start of the list val shuffledTopicPartitions3 = Seq(partitionWithLargeMessage1, partitionWithLargeMessage2) ++ random.shuffle(partitionsWithoutLargeMessages) - val fetchRequest3 = createFetchRequest(shuffledTopicPartitions3, Map(partitionWithLargeMessage1 -> messagesPerPartition)) + val fetchRequest3 = createConsumerFetchRequest(shuffledTopicPartitions3, Map(partitionWithLargeMessage1 -> messagesPerPartition)) val fetchResponse3 = sendFetchRequest(leaderId, fetchRequest3) - val fetchRequest3V12 = createFetchRequest(shuffledTopicPartitions3, Map(partitionWithLargeMessage1 -> messagesPerPartition), 12) + val fetchRequest3V12 = createConsumerFetchRequest(shuffledTopicPartitions3, Map(partitionWithLargeMessage1 -> messagesPerPartition), 12) val fetchResponse3V12 = sendFetchRequest(leaderId, fetchRequest3V12) def evaluateResponse3(response: FetchResponse, version: Short = ApiKeys.FETCH.latestVersion()) = { val responseData = response.responseData(topicNames, version) @@ -121,9 +121,9 @@ class FetchRequestTest extends BaseFetchRequestTest { // 4. Partition with message larger than the response limit at the start of the list val shuffledTopicPartitions4 = Seq(partitionWithLargeMessage2, partitionWithLargeMessage1) ++ random.shuffle(partitionsWithoutLargeMessages) - val fetchRequest4 = createFetchRequest(shuffledTopicPartitions4, Map(partitionWithLargeMessage2 -> messagesPerPartition)) + val fetchRequest4 = createConsumerFetchRequest(shuffledTopicPartitions4, Map(partitionWithLargeMessage2 -> messagesPerPartition)) val fetchResponse4 = sendFetchRequest(leaderId, fetchRequest4) - val fetchRequest4V12 = createFetchRequest(shuffledTopicPartitions4, Map(partitionWithLargeMessage2 -> messagesPerPartition), 12) + val fetchRequest4V12 = createConsumerFetchRequest(shuffledTopicPartitions4, Map(partitionWithLargeMessage2 -> messagesPerPartition), 12) val fetchResponse4V12 = sendFetchRequest(leaderId, fetchRequest4V12) def evaluateResponse4(response: FetchResponse, version: Short = ApiKeys.FETCH.latestVersion()) = { val responseData = response.responseData(topicNames, version) @@ -507,7 +507,7 @@ class FetchRequestTest extends BaseFetchRequestTest { */ @Test def testCreateIncrementalFetchWithPartitionsInErrorV12(): Unit = { - def createFetchRequest(topicPartitions: Seq[TopicPartition], + def createConsumerFetchRequest(topicPartitions: Seq[TopicPartition], metadata: JFetchMetadata, toForget: Seq[TopicIdPartition]): FetchRequest = FetchRequest.Builder.forConsumer(12, Int.MaxValue, 0, @@ -521,7 +521,7 @@ class FetchRequestTest extends BaseFetchRequestTest { val topicNames = Map[Uuid, String]().asJava createTopicWithAssignment("foo", Map(0 -> List(0, 1), 1 -> List(0, 2))) val bar0 = new TopicPartition("bar", 0) - val req1 = createFetchRequest(List(foo0, foo1, bar0), JFetchMetadata.INITIAL, Nil) + val req1 = createConsumerFetchRequest(List(foo0, foo1, bar0), JFetchMetadata.INITIAL, Nil) val resp1 = sendFetchRequest(0, req1) assertEquals(Errors.NONE, resp1.error()) assertTrue(resp1.sessionId() > 0, "Expected the broker to create a new incremental fetch session") @@ -533,7 +533,7 @@ class FetchRequestTest extends BaseFetchRequestTest { assertEquals(Errors.NONE.code, responseData1.get(foo0).errorCode) assertEquals(Errors.NONE.code, responseData1.get(foo1).errorCode) assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, responseData1.get(bar0).errorCode) - val req2 = createFetchRequest(Nil, new JFetchMetadata(resp1.sessionId(), 1), Nil) + val req2 = createConsumerFetchRequest(Nil, new JFetchMetadata(resp1.sessionId(), 1), Nil) val resp2 = sendFetchRequest(0, req2) assertEquals(Errors.NONE, resp2.error()) assertEquals(resp1.sessionId(), @@ -544,7 +544,7 @@ class FetchRequestTest extends BaseFetchRequestTest { assertTrue(responseData2.containsKey(bar0)) assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, responseData2.get(bar0).errorCode) createTopicWithAssignment("bar", Map(0 -> List(0, 1))) - val req3 = createFetchRequest(Nil, new JFetchMetadata(resp1.sessionId(), 2), Nil) + val req3 = createConsumerFetchRequest(Nil, new JFetchMetadata(resp1.sessionId(), 2), Nil) val resp3 = sendFetchRequest(0, req3) assertEquals(Errors.NONE, resp3.error()) val responseData3 = resp3.responseData(topicNames, 12) @@ -552,7 +552,7 @@ class FetchRequestTest extends BaseFetchRequestTest { assertFalse(responseData3.containsKey(foo1)) assertTrue(responseData3.containsKey(bar0)) assertEquals(Errors.NONE.code, responseData3.get(bar0).errorCode) - val req4 = createFetchRequest(Nil, new JFetchMetadata(resp1.sessionId(), 3), Nil) + val req4 = createConsumerFetchRequest(Nil, new JFetchMetadata(resp1.sessionId(), 3), Nil) val resp4 = sendFetchRequest(0, req4) assertEquals(Errors.NONE, resp4.error()) val responseData4 = resp4.responseData(topicNames, 12) @@ -566,7 +566,7 @@ class FetchRequestTest extends BaseFetchRequestTest { */ @Test def testFetchWithPartitionsWithIdError(): Unit = { - def createFetchRequest(fetchData: util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData], + def createConsumerFetchRequest(fetchData: util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData], metadata: JFetchMetadata, toForget: Seq[TopicIdPartition]): FetchRequest = { FetchRequest.Builder.forConsumer(ApiKeys.FETCH.latestVersion(), Int.MaxValue, 0, fetchData) @@ -593,7 +593,7 @@ class FetchRequestTest extends BaseFetchRequestTest { partitionMap } - val req1 = createFetchRequest( createPartitionMap(Integer.MAX_VALUE, List(foo0, foo1, bar0), Map.empty), JFetchMetadata.INITIAL, Nil) + val req1 = createConsumerFetchRequest( createPartitionMap(Integer.MAX_VALUE, List(foo0, foo1, bar0), Map.empty), JFetchMetadata.INITIAL, Nil) val resp1 = sendFetchRequest(0, req1) assertEquals(Errors.NONE, resp1.error()) val topicNames1 = topicIdsWithUnknown.map(_.swap).asJava diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index d176f369f8df4..d0e5687f67a62 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -23,7 +23,6 @@ import java.util import java.util.Arrays.asList import java.util.concurrent.TimeUnit import java.util.{Collections, Optional, Properties, Random} - import kafka.api.LeaderAndIsr import kafka.cluster.Broker import kafka.controller.{ControllerContext, KafkaController} @@ -84,7 +83,7 @@ import org.apache.kafka.server.authorizer.{Action, AuthorizationResult, Authoriz import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.params.provider.{CsvSource, ValueSource} import org.mockito.ArgumentMatchers.{any, anyBoolean, anyDouble, anyInt, anyLong, anyShort, anyString, argThat, isNotNull} import org.mockito.Mockito.{mock, reset, times, verify, when} import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito} @@ -92,6 +91,7 @@ import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito} import scala.collection.{Map, Seq, mutable} import scala.jdk.CollectionConverters._ import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic +import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1} @@ -181,7 +181,7 @@ class KafkaApisTest { } else { ApiKeys.apisForListener(listenerType).asScala.toSet } - val apiVersionManager = new SimpleApiVersionManager(listenerType, enabledApis) + val apiVersionManager = new SimpleApiVersionManager(listenerType, enabledApis, BrokerFeatures.defaultSupportedFeatures()) new KafkaApis( metadataSupport = metadataSupport, @@ -772,6 +772,57 @@ class KafkaApisTest { testForwardableApi(ApiKeys.CREATE_TOPICS, requestBuilder) } + @ParameterizedTest + @CsvSource(value = Array("0,1500", "1500,0", "3000,1000")) + def testKRaftControllerThrottleTimeEnforced( + controllerThrottleTimeMs: Int, + requestThrottleTimeMs: Int + ): Unit = { + metadataCache = MetadataCache.kRaftMetadataCache(brokerId) + + val topicToCreate = new CreatableTopic() + .setName("topic") + .setNumPartitions(1) + .setReplicationFactor(1.toShort) + + val requestData = new CreateTopicsRequestData() + requestData.topics().add(topicToCreate) + + val requestBuilder = new CreateTopicsRequest.Builder(requestData).build() + val request = buildRequest(requestBuilder) + + val kafkaApis = createKafkaApis(enableForwarding = true, raftSupport = true) + val forwardCallback: ArgumentCaptor[Option[AbstractResponse] => Unit] = + ArgumentCaptor.forClass(classOf[Option[AbstractResponse] => Unit]) + + when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(request, time.milliseconds())) + .thenReturn(requestThrottleTimeMs) + + kafkaApis.handle(request, RequestLocal.withThreadConfinedCaching) + + verify(forwardingManager).forwardRequest( + ArgumentMatchers.eq(request), + forwardCallback.capture() + ) + + val responseData = new CreateTopicsResponseData() + .setThrottleTimeMs(controllerThrottleTimeMs) + responseData.topics().add(new CreatableTopicResult() + .setErrorCode(Errors.THROTTLING_QUOTA_EXCEEDED.code)) + + forwardCallback.getValue.apply(Some(new CreateTopicsResponse(responseData))) + + val expectedThrottleTimeMs = math.max(controllerThrottleTimeMs, requestThrottleTimeMs) + + verify(clientRequestQuotaManager).throttle( + ArgumentMatchers.eq(request), + any[ThrottleCallback](), + ArgumentMatchers.eq(expectedThrottleTimeMs) + ) + + assertEquals(expectedThrottleTimeMs, responseData.throttleTimeMs) + } + @Test def testCreatePartitionsAuthorization(): Unit = { val authorizer: Authorizer = mock(classOf[Authorizer]) @@ -923,6 +974,12 @@ class KafkaApisTest { testForwardableApi(ApiKeys.CREATE_PARTITIONS, requestBuilder) } + @Test + def testUpdateFeaturesWithForwarding(): Unit = { + val requestBuilder = new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData()) + testForwardableApi(ApiKeys.UPDATE_FEATURES, requestBuilder) + } + @Test def testDeleteTopicsWithForwarding(): Unit = { val requestBuilder = new DeleteTopicsRequest.Builder(new DeleteTopicsRequestData()) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index ee638ba893d97..b9eee175503b5 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -49,7 +49,7 @@ class KafkaConfigTest { @Test def testLogRetentionTimeHoursProvided(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.LogRetentionTimeHoursProp, "1") + props.setProperty(KafkaConfig.LogRetentionTimeHoursProp, "1") val cfg = KafkaConfig.fromProps(props) assertEquals(60L * 60L * 1000L, cfg.logRetentionTimeMillis) @@ -58,7 +58,7 @@ class KafkaConfigTest { @Test def testLogRetentionTimeMinutesProvided(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.LogRetentionTimeMinutesProp, "30") + props.setProperty(KafkaConfig.LogRetentionTimeMinutesProp, "30") val cfg = KafkaConfig.fromProps(props) assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis) @@ -67,7 +67,7 @@ class KafkaConfigTest { @Test def testLogRetentionTimeMsProvided(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.LogRetentionTimeMillisProp, "1800000") + props.setProperty(KafkaConfig.LogRetentionTimeMillisProp, "1800000") val cfg = KafkaConfig.fromProps(props) assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis) @@ -84,8 +84,8 @@ class KafkaConfigTest { @Test def testLogRetentionTimeBothMinutesAndHoursProvided(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.LogRetentionTimeMinutesProp, "30") - props.put(KafkaConfig.LogRetentionTimeHoursProp, "1") + props.setProperty(KafkaConfig.LogRetentionTimeMinutesProp, "30") + props.setProperty(KafkaConfig.LogRetentionTimeHoursProp, "1") val cfg = KafkaConfig.fromProps(props) assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis) @@ -94,8 +94,8 @@ class KafkaConfigTest { @Test def testLogRetentionTimeBothMinutesAndMsProvided(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.LogRetentionTimeMillisProp, "1800000") - props.put(KafkaConfig.LogRetentionTimeMinutesProp, "10") + props.setProperty(KafkaConfig.LogRetentionTimeMillisProp, "1800000") + props.setProperty(KafkaConfig.LogRetentionTimeMinutesProp, "10") val cfg = KafkaConfig.fromProps(props) assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis) @@ -109,9 +109,9 @@ class KafkaConfigTest { val props4 = TestUtils.createBrokerConfig(0,TestUtils.MockZkConnect, port = 8181) val props5 = TestUtils.createBrokerConfig(0,TestUtils.MockZkConnect, port = 8181) - props1.put("log.retention.ms", "-1") - props2.put("log.retention.minutes", "-1") - props3.put("log.retention.hours", "-1") + props1.setProperty("log.retention.ms", "-1") + props2.setProperty("log.retention.minutes", "-1") + props3.setProperty("log.retention.hours", "-1") val cfg1 = KafkaConfig.fromProps(props1) val cfg2 = KafkaConfig.fromProps(props2) @@ -120,13 +120,13 @@ class KafkaConfigTest { assertEquals(-1, cfg2.logRetentionTimeMillis, "Should be -1") assertEquals(-1, cfg3.logRetentionTimeMillis, "Should be -1") - props4.put("log.retention.ms", "-1") - props4.put("log.retention.minutes", "30") + props4.setProperty("log.retention.ms", "-1") + props4.setProperty("log.retention.minutes", "30") val cfg4 = KafkaConfig.fromProps(props4) assertEquals(-1, cfg4.logRetentionTimeMillis, "Should be -1") - props5.put("log.retention.ms", "0") + props5.setProperty("log.retention.ms", "0") assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props5)) } @@ -137,9 +137,9 @@ class KafkaConfigTest { val props2 = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props3 = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props1.put("log.retention.ms", "0") - props2.put("log.retention.minutes", "0") - props3.put("log.retention.hours", "0") + props1.setProperty("log.retention.ms", "0") + props2.setProperty("log.retention.minutes", "0") + props3.setProperty("log.retention.hours", "0") assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props1)) assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props2)) @@ -152,9 +152,9 @@ class KafkaConfigTest { val port = 9999 val hostName = "fake-host" val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") - props.put(KafkaConfig.ZkConnectProp, "localhost:2181") - props.put(KafkaConfig.ListenersProp, s"PLAINTEXT://$hostName:$port") + props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaConfig.ZkConnectProp, "localhost:2181") + props.setProperty(KafkaConfig.ListenersProp, s"PLAINTEXT://$hostName:$port") val serverConfig = KafkaConfig.fromProps(props) val endpoints = serverConfig.effectiveAdvertisedListeners @@ -170,7 +170,7 @@ class KafkaConfigTest { val advertisedPort = 1234 val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) - props.put(KafkaConfig.AdvertisedListenersProp, s"PLAINTEXT://$advertisedHostName:$advertisedPort") + props.setProperty(KafkaConfig.AdvertisedListenersProp, s"PLAINTEXT://$advertisedHostName:$advertisedPort") val serverConfig = KafkaConfig.fromProps(props) val endpoints = serverConfig.effectiveAdvertisedListeners @@ -183,35 +183,35 @@ class KafkaConfigTest { @Test def testDuplicateListeners(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") - props.put(KafkaConfig.ZkConnectProp, "localhost:2181") + props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaConfig.ZkConnectProp, "localhost:2181") // listeners with duplicate port - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9091,SSL://localhost:9091") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9091,SSL://localhost:9091") assertBadConfigContainingMessage(props, "Each listener must have a different port") // listeners with duplicate name - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9091,PLAINTEXT://localhost:9092") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9091,PLAINTEXT://localhost:9092") assertBadConfigContainingMessage(props, "Each listener must have a different name") // advertised listeners can have duplicate ports - props.put(KafkaConfig.ListenerSecurityProtocolMapProp, "HOST:SASL_SSL,LB:SASL_SSL") - props.put(KafkaConfig.InterBrokerListenerNameProp, "HOST") - props.put(KafkaConfig.ListenersProp, "HOST://localhost:9091,LB://localhost:9092") - props.put(KafkaConfig.AdvertisedListenersProp, "HOST://localhost:9091,LB://localhost:9091") + props.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, "HOST:SASL_SSL,LB:SASL_SSL") + props.setProperty(KafkaConfig.InterBrokerListenerNameProp, "HOST") + props.setProperty(KafkaConfig.ListenersProp, "HOST://localhost:9091,LB://localhost:9092") + props.setProperty(KafkaConfig.AdvertisedListenersProp, "HOST://localhost:9091,LB://localhost:9091") KafkaConfig.fromProps(props) // but not duplicate names - props.put(KafkaConfig.AdvertisedListenersProp, "HOST://localhost:9091,HOST://localhost:9091") + props.setProperty(KafkaConfig.AdvertisedListenersProp, "HOST://localhost:9091,HOST://localhost:9091") assertBadConfigContainingMessage(props, "Each listener must have a different name") } @Test def testControlPlaneListenerName(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) - props.put("listeners", "PLAINTEXT://localhost:0,CONTROLLER://localhost:5000") - props.put("listener.security.protocol.map", "PLAINTEXT:PLAINTEXT,CONTROLLER:SSL") - props.put("control.plane.listener.name", "CONTROLLER") + props.setProperty("listeners", "PLAINTEXT://localhost:0,CONTROLLER://localhost:5000") + props.setProperty("listener.security.protocol.map", "PLAINTEXT:PLAINTEXT,CONTROLLER:SSL") + props.setProperty("control.plane.listener.name", "CONTROLLER") KafkaConfig.fromProps(props) val serverConfig = KafkaConfig.fromProps(props) @@ -234,12 +234,12 @@ class KafkaConfigTest { @Test def testControllerListenerNames(): Unit = { val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "broker,controller") - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:0,CONTROLLER://localhost:5000") - props.put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") - props.put(KafkaConfig.NodeIdProp, "2") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:5000") - props.put(KafkaConfig.ListenerSecurityProtocolMapProp, "PLAINTEXT:PLAINTEXT,CONTROLLER:SASL_SSL") + props.setProperty(KafkaConfig.ProcessRolesProp, "broker,controller") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:0,CONTROLLER://localhost:5000") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") + props.setProperty(KafkaConfig.NodeIdProp, "2") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:5000") + props.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, "PLAINTEXT:PLAINTEXT,CONTROLLER:SASL_SSL") val serverConfig = KafkaConfig.fromProps(props) val controllerEndpoints = serverConfig.controllerListeners @@ -253,12 +253,12 @@ class KafkaConfigTest { @Test def testControlPlaneListenerNameNotAllowedWithKRaft(): Unit = { val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "broker,controller") - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093") - props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") - props.put(KafkaConfig.NodeIdProp, "2") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9093") - props.put(KafkaConfig.ControlPlaneListenerNameProp, "SSL") + props.setProperty(KafkaConfig.ProcessRolesProp, "broker,controller") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") + props.setProperty(KafkaConfig.NodeIdProp, "2") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") + props.setProperty(KafkaConfig.ControlPlaneListenerNameProp, "SSL") assertFalse(isValidKafkaConfig(props)) assertBadConfigContainingMessage(props, "control.plane.listener.name is not supported in KRaft mode.") @@ -270,119 +270,119 @@ class KafkaConfigTest { @Test def testControllerListenerDefinedForKRaftController(): Unit = { val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "controller") - props.put(KafkaConfig.ListenersProp, "SSL://localhost:9093") - props.put(KafkaConfig.NodeIdProp, "2") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9093") + props.setProperty(KafkaConfig.ProcessRolesProp, "controller") + props.setProperty(KafkaConfig.ListenersProp, "SSL://localhost:9093") + props.setProperty(KafkaConfig.NodeIdProp, "2") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") assertBadConfigContainingMessage(props, "The listeners config must only contain KRaft controller listeners from controller.listener.names when process.roles=controller") - props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") KafkaConfig.fromProps(props) // confirm that redirecting via listener.security.protocol.map is acceptable - props.put(KafkaConfig.ListenerSecurityProtocolMapProp, "CONTROLLER:SSL") - props.put(KafkaConfig.ListenersProp, "CONTROLLER://localhost:9093") - props.put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") + props.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, "CONTROLLER:SSL") + props.setProperty(KafkaConfig.ListenersProp, "CONTROLLER://localhost:9093") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") KafkaConfig.fromProps(props) } @Test def testControllerListenerDefinedForKRaftBroker(): Unit = { val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "broker") - props.put(KafkaConfig.NodeIdProp, "1") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9093") + props.setProperty(KafkaConfig.ProcessRolesProp, "broker") + props.setProperty(KafkaConfig.NodeIdProp, "1") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") assertFalse(isValidKafkaConfig(props)) assertBadConfigContainingMessage(props, "controller.listener.names must contain at least one value when running KRaft with just the broker role") - props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") KafkaConfig.fromProps(props) // confirm that redirecting via listener.security.protocol.map is acceptable - props.put(KafkaConfig.ListenerSecurityProtocolMapProp, "PLAINTEXT:PLAINTEXT,CONTROLLER:SSL") - props.put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") + props.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, "PLAINTEXT:PLAINTEXT,CONTROLLER:SSL") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") KafkaConfig.fromProps(props) } @Test def testPortInQuorumVotersNotRequiredToMatchFirstControllerListenerPortForThisKRaftController(): Unit = { val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "controller,broker") - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093,SASL_SSL://localhost:9094") - props.put(KafkaConfig.NodeIdProp, "2") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9093,3@anotherhost:9094") - props.put(KafkaConfig.ControllerListenerNamesProp, "SSL,SASL_SSL") + props.setProperty(KafkaConfig.ProcessRolesProp, "controller,broker") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093,SASL_SSL://localhost:9094") + props.setProperty(KafkaConfig.NodeIdProp, "2") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093,3@anotherhost:9094") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL,SASL_SSL") KafkaConfig.fromProps(props) // change each of the 4 ports to port 5555 -- should pass in all circumstances since we can't validate the // controller.quorum.voters ports (which are the ports that clients use and are semantically "advertised" ports // even though the controller configuration doesn't list them in advertised.listeners) against the // listener ports (which are semantically different then the ports that clients use). - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:5555,SASL_SSL://localhost:9094") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:5555,SASL_SSL://localhost:9094") KafkaConfig.fromProps(props) - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093,SASL_SSL://localhost:5555") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093,SASL_SSL://localhost:5555") KafkaConfig.fromProps(props) - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093,SASL_SSL://localhost:9094") // reset to original value - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:5555,3@anotherhost:9094") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093,SASL_SSL://localhost:9094") // reset to original value + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:5555,3@anotherhost:9094") KafkaConfig.fromProps(props) - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9093,3@anotherhost:5555") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093,3@anotherhost:5555") KafkaConfig.fromProps(props) } @Test def testSeparateControllerListenerDefinedForKRaftBrokerController(): Unit = { val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "broker,controller") - props.put(KafkaConfig.ListenersProp, "SSL://localhost:9093") - props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") - props.put(KafkaConfig.NodeIdProp, "2") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9093") + props.setProperty(KafkaConfig.ProcessRolesProp, "broker,controller") + props.setProperty(KafkaConfig.ListenersProp, "SSL://localhost:9093") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") + props.setProperty(KafkaConfig.NodeIdProp, "2") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") assertFalse(isValidKafkaConfig(props)) assertBadConfigContainingMessage(props, "There must be at least one advertised listener. Perhaps all listeners appear in controller.listener.names?") - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093") KafkaConfig.fromProps(props) // confirm that redirecting via listener.security.protocol.map is acceptable - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,CONTROLLER://localhost:9093") - props.put(KafkaConfig.ListenerSecurityProtocolMapProp, "PLAINTEXT:PLAINTEXT,CONTROLLER:SSL") - props.put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,CONTROLLER://localhost:9093") + props.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, "PLAINTEXT:PLAINTEXT,CONTROLLER:SSL") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") KafkaConfig.fromProps(props) } @Test def testControllerListenerNameMapsToPlaintextByDefaultForKRaft(): Unit = { val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "broker") - props.put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") - props.put(KafkaConfig.NodeIdProp, "1") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9093") + props.setProperty(KafkaConfig.ProcessRolesProp, "broker") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") + props.setProperty(KafkaConfig.NodeIdProp, "1") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") val controllerListenerName = new ListenerName("CONTROLLER") assertEquals(Some(SecurityProtocol.PLAINTEXT), KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(controllerListenerName)) // ensure we don't map it to PLAINTEXT when there is a SSL or SASL controller listener - props.put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER,SSL") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER,SSL") val controllerNotFoundInMapMessage = "Controller listener with name CONTROLLER defined in controller.listener.names not found in listener.security.protocol.map" assertBadConfigContainingMessage(props, controllerNotFoundInMapMessage) // ensure we don't map it to PLAINTEXT when there is a SSL or SASL listener - props.put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") - props.put(KafkaConfig.ListenersProp, "SSL://localhost:9092") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") + props.setProperty(KafkaConfig.ListenersProp, "SSL://localhost:9092") assertBadConfigContainingMessage(props, controllerNotFoundInMapMessage) props.remove(KafkaConfig.ListenersProp) // ensure we don't map it to PLAINTEXT when it is explicitly mapped otherwise - props.put(KafkaConfig.ListenerSecurityProtocolMapProp, "PLAINTEXT:PLAINTEXT,CONTROLLER:SSL") + props.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, "PLAINTEXT:PLAINTEXT,CONTROLLER:SSL") assertEquals(Some(SecurityProtocol.SSL), KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(controllerListenerName)) // ensure we don't map it to PLAINTEXT when anything is explicitly given // (i.e. it is only part of the default value, even with KRaft) - props.put(KafkaConfig.ListenerSecurityProtocolMapProp, "PLAINTEXT:PLAINTEXT") + props.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, "PLAINTEXT:PLAINTEXT") assertBadConfigContainingMessage(props, controllerNotFoundInMapMessage) // ensure we can map it to a non-PLAINTEXT security protocol by default (i.e. when nothing is given) props.remove(KafkaConfig.ListenerSecurityProtocolMapProp) - props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") assertEquals(Some(SecurityProtocol.SSL), KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("SSL"))) } @@ -390,11 +390,11 @@ class KafkaConfigTest { @Test def testMultipleControllerListenerNamesMapToPlaintextByDefaultForKRaft(): Unit = { val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "controller") - props.put(KafkaConfig.ListenersProp, "CONTROLLER1://localhost:9092,CONTROLLER2://localhost:9093") - props.put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER1,CONTROLLER2") - props.put(KafkaConfig.NodeIdProp, "1") - props.put(KafkaConfig.QuorumVotersProp, "1@localhost:9092") + props.setProperty(KafkaConfig.ProcessRolesProp, "controller") + props.setProperty(KafkaConfig.ListenersProp, "CONTROLLER1://localhost:9092,CONTROLLER2://localhost:9093") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER1,CONTROLLER2") + props.setProperty(KafkaConfig.NodeIdProp, "1") + props.setProperty(KafkaConfig.QuorumVotersProp, "1@localhost:9092") assertEquals(Some(SecurityProtocol.PLAINTEXT), KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("CONTROLLER1"))) assertEquals(Some(SecurityProtocol.PLAINTEXT), @@ -404,22 +404,22 @@ class KafkaConfigTest { @Test def testControllerListenerNameDoesNotMapToPlaintextByDefaultForNonKRaft(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") - props.put(KafkaConfig.ZkConnectProp, "localhost:2181") - props.put(KafkaConfig.ListenersProp, "CONTROLLER://localhost:9092") + props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaConfig.ZkConnectProp, "localhost:2181") + props.setProperty(KafkaConfig.ListenersProp, "CONTROLLER://localhost:9092") assertBadConfigContainingMessage(props, "Error creating broker listeners from 'CONTROLLER://localhost:9092': No security protocol defined for listener CONTROLLER") // Valid now - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092") assertEquals(None, KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("CONTROLLER"))) } @Test def testBadListenerProtocol(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") - props.put(KafkaConfig.ZkConnectProp, "localhost:2181") - props.put(KafkaConfig.ListenersProp, "BAD://localhost:9091") + props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaConfig.ZkConnectProp, "localhost:2181") + props.setProperty(KafkaConfig.ListenersProp, "BAD://localhost:9091") assertFalse(isValidKafkaConfig(props)) } @@ -427,12 +427,12 @@ class KafkaConfigTest { @Test def testListenerNamesWithAdvertisedListenerUnset(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") - props.put(KafkaConfig.ZkConnectProp, "localhost:2181") + props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaConfig.ZkConnectProp, "localhost:2181") - props.put(KafkaConfig.ListenersProp, "CLIENT://localhost:9091,REPLICATION://localhost:9092,INTERNAL://localhost:9093") - props.put(KafkaConfig.ListenerSecurityProtocolMapProp, "CLIENT:SSL,REPLICATION:SSL,INTERNAL:PLAINTEXT") - props.put(KafkaConfig.InterBrokerListenerNameProp, "REPLICATION") + props.setProperty(KafkaConfig.ListenersProp, "CLIENT://localhost:9091,REPLICATION://localhost:9092,INTERNAL://localhost:9093") + props.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, "CLIENT:SSL,REPLICATION:SSL,INTERNAL:PLAINTEXT") + props.setProperty(KafkaConfig.InterBrokerListenerNameProp, "REPLICATION") val config = KafkaConfig.fromProps(props) val expectedListeners = Seq( EndPoint("localhost", 9091, new ListenerName("CLIENT"), SecurityProtocol.SSL), @@ -451,13 +451,13 @@ class KafkaConfigTest { @Test def testListenerAndAdvertisedListenerNames(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") - props.put(KafkaConfig.ZkConnectProp, "localhost:2181") + props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaConfig.ZkConnectProp, "localhost:2181") - props.put(KafkaConfig.ListenersProp, "EXTERNAL://localhost:9091,INTERNAL://localhost:9093") - props.put(KafkaConfig.AdvertisedListenersProp, "EXTERNAL://lb1.example.com:9000,INTERNAL://host1:9093") - props.put(KafkaConfig.ListenerSecurityProtocolMapProp, "EXTERNAL:SSL,INTERNAL:PLAINTEXT") - props.put(KafkaConfig.InterBrokerListenerNameProp, "INTERNAL") + props.setProperty(KafkaConfig.ListenersProp, "EXTERNAL://localhost:9091,INTERNAL://localhost:9093") + props.setProperty(KafkaConfig.AdvertisedListenersProp, "EXTERNAL://lb1.example.com:9000,INTERNAL://host1:9093") + props.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, "EXTERNAL:SSL,INTERNAL:PLAINTEXT") + props.setProperty(KafkaConfig.InterBrokerListenerNameProp, "INTERNAL") val config = KafkaConfig.fromProps(props) val expectedListeners = Seq( @@ -482,43 +482,43 @@ class KafkaConfigTest { @Test def testListenerNameMissingFromListenerSecurityProtocolMap(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") - props.put(KafkaConfig.ZkConnectProp, "localhost:2181") + props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaConfig.ZkConnectProp, "localhost:2181") - props.put(KafkaConfig.ListenersProp, "SSL://localhost:9091,REPLICATION://localhost:9092") - props.put(KafkaConfig.InterBrokerListenerNameProp, "SSL") + props.setProperty(KafkaConfig.ListenersProp, "SSL://localhost:9091,REPLICATION://localhost:9092") + props.setProperty(KafkaConfig.InterBrokerListenerNameProp, "SSL") assertFalse(isValidKafkaConfig(props)) } @Test def testInterBrokerListenerNameMissingFromListenerSecurityProtocolMap(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") - props.put(KafkaConfig.ZkConnectProp, "localhost:2181") + props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaConfig.ZkConnectProp, "localhost:2181") - props.put(KafkaConfig.ListenersProp, "SSL://localhost:9091") - props.put(KafkaConfig.InterBrokerListenerNameProp, "REPLICATION") + props.setProperty(KafkaConfig.ListenersProp, "SSL://localhost:9091") + props.setProperty(KafkaConfig.InterBrokerListenerNameProp, "REPLICATION") assertFalse(isValidKafkaConfig(props)) } @Test def testInterBrokerListenerNameAndSecurityProtocolSet(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") - props.put(KafkaConfig.ZkConnectProp, "localhost:2181") + props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaConfig.ZkConnectProp, "localhost:2181") - props.put(KafkaConfig.ListenersProp, "SSL://localhost:9091") - props.put(KafkaConfig.InterBrokerListenerNameProp, "SSL") - props.put(KafkaConfig.InterBrokerSecurityProtocolProp, "SSL") + props.setProperty(KafkaConfig.ListenersProp, "SSL://localhost:9091") + props.setProperty(KafkaConfig.InterBrokerListenerNameProp, "SSL") + props.setProperty(KafkaConfig.InterBrokerSecurityProtocolProp, "SSL") assertFalse(isValidKafkaConfig(props)) } @Test def testCaseInsensitiveListenerProtocol(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") - props.put(KafkaConfig.ZkConnectProp, "localhost:2181") - props.put(KafkaConfig.ListenersProp, "plaintext://localhost:9091,SsL://localhost:9092") + props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaConfig.ZkConnectProp, "localhost:2181") + props.setProperty(KafkaConfig.ListenersProp, "plaintext://localhost:9091,SsL://localhost:9092") val config = KafkaConfig.fromProps(props) assertEquals(Some("SSL://localhost:9092"), config.listeners.find(_.listenerName.value == "SSL").map(_.connectionString)) assertEquals(Some("PLAINTEXT://localhost:9091"), config.listeners.find(_.listenerName.value == "PLAINTEXT").map(_.connectionString)) @@ -531,8 +531,8 @@ class KafkaConfigTest { @Test def testListenerDefaults(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") - props.put(KafkaConfig.ZkConnectProp, "localhost:2181") + props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaConfig.ZkConnectProp, "localhost:2181") // configuration with no listeners val conf = KafkaConfig.fromProps(props) @@ -545,21 +545,21 @@ class KafkaConfigTest { @Test def testVersionConfiguration(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") - props.put(KafkaConfig.ZkConnectProp, "localhost:2181") + props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaConfig.ZkConnectProp, "localhost:2181") val conf = KafkaConfig.fromProps(props) assertEquals(MetadataVersion.latest, conf.interBrokerProtocolVersion) - props.put(KafkaConfig.InterBrokerProtocolVersionProp, "0.8.2.0") + props.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, "0.8.2.0") // We need to set the message format version to make the configuration valid. - props.put(KafkaConfig.LogMessageFormatVersionProp, "0.8.2.0") + props.setProperty(KafkaConfig.LogMessageFormatVersionProp, "0.8.2.0") val conf2 = KafkaConfig.fromProps(props) assertEquals(IBP_0_8_2, conf2.interBrokerProtocolVersion) // check that 0.8.2.0 is the same as 0.8.2.1 - props.put(KafkaConfig.InterBrokerProtocolVersionProp, "0.8.2.1") + props.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, "0.8.2.1") // We need to set the message format version to make the configuration valid - props.put(KafkaConfig.LogMessageFormatVersionProp, "0.8.2.1") + props.setProperty(KafkaConfig.LogMessageFormatVersionProp, "0.8.2.1") val conf3 = KafkaConfig.fromProps(props) assertEquals(IBP_0_8_2, conf3.interBrokerProtocolVersion) @@ -587,7 +587,7 @@ class KafkaConfigTest { @Test def testUncleanElectionDisabled(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.UncleanLeaderElectionEnableProp, String.valueOf(false)) + props.setProperty(KafkaConfig.UncleanLeaderElectionEnableProp, String.valueOf(false)) val serverConfig = KafkaConfig.fromProps(props) assertEquals(serverConfig.uncleanLeaderElectionEnable, false) @@ -596,7 +596,7 @@ class KafkaConfigTest { @Test def testUncleanElectionEnabled(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.UncleanLeaderElectionEnableProp, String.valueOf(true)) + props.setProperty(KafkaConfig.UncleanLeaderElectionEnableProp, String.valueOf(true)) val serverConfig = KafkaConfig.fromProps(props) assertEquals(serverConfig.uncleanLeaderElectionEnable, true) @@ -605,7 +605,7 @@ class KafkaConfigTest { @Test def testUncleanElectionInvalid(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.UncleanLeaderElectionEnableProp, "invalid") + props.setProperty(KafkaConfig.UncleanLeaderElectionEnableProp, "invalid") assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @@ -613,7 +613,7 @@ class KafkaConfigTest { @Test def testLogRollTimeMsProvided(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.LogRollTimeMillisProp, "1800000") + props.setProperty(KafkaConfig.LogRollTimeMillisProp, "1800000") val cfg = KafkaConfig.fromProps(props) assertEquals(30 * 60L * 1000L, cfg.logRollTimeMillis) @@ -622,8 +622,8 @@ class KafkaConfigTest { @Test def testLogRollTimeBothMsAndHoursProvided(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.LogRollTimeMillisProp, "1800000") - props.put(KafkaConfig.LogRollTimeHoursProp, "1") + props.setProperty(KafkaConfig.LogRollTimeMillisProp, "1800000") + props.setProperty(KafkaConfig.LogRollTimeHoursProp, "1") val cfg = KafkaConfig.fromProps(props) assertEquals( 30 * 60L * 1000L, cfg.logRollTimeMillis) @@ -648,7 +648,7 @@ class KafkaConfigTest { @Test def testValidCompressionType(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put("compression.type", "gzip") + props.setProperty("compression.type", "gzip") val serverConfig = KafkaConfig.fromProps(props) assertEquals(serverConfig.compressionType, "gzip") @@ -657,34 +657,34 @@ class KafkaConfigTest { @Test def testInvalidCompressionType(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.CompressionTypeProp, "abc") + props.setProperty(KafkaConfig.CompressionTypeProp, "abc") assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @Test def testInvalidInterBrokerSecurityProtocol(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.ListenersProp, "SSL://localhost:0") - props.put(KafkaConfig.InterBrokerSecurityProtocolProp, SecurityProtocol.PLAINTEXT.toString) + props.setProperty(KafkaConfig.ListenersProp, "SSL://localhost:0") + props.setProperty(KafkaConfig.InterBrokerSecurityProtocolProp, SecurityProtocol.PLAINTEXT.toString) assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)) } @Test def testEqualAdvertisedListenersProtocol(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093") - props.put(KafkaConfig.AdvertisedListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093") + props.setProperty(KafkaConfig.AdvertisedListenersProp, "PLAINTEXT://localhost:9092,SSL://localhost:9093") KafkaConfig.fromProps(props) } @Test def testInvalidAdvertisedListenersProtocol(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.ListenersProp, "TRACE://localhost:9091,SSL://localhost:9093") - props.put(KafkaConfig.AdvertisedListenersProp, "PLAINTEXT://localhost:9092") + props.setProperty(KafkaConfig.ListenersProp, "TRACE://localhost:9091,SSL://localhost:9093") + props.setProperty(KafkaConfig.AdvertisedListenersProp, "PLAINTEXT://localhost:9092") assertBadConfigContainingMessage(props, "No security protocol defined for listener TRACE") - props.put(KafkaConfig.ListenerSecurityProtocolMapProp, "PLAINTEXT:PLAINTEXT,TRACE:PLAINTEXT,SSL:SSL") + props.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, "PLAINTEXT:PLAINTEXT,TRACE:PLAINTEXT,SSL:SSL") assertBadConfigContainingMessage(props, "advertised.listeners listener names must be equal to or a subset of the ones defined in listeners") } @@ -693,8 +693,8 @@ class KafkaConfigTest { def testInterBrokerVersionMessageFormatCompatibility(): Unit = { def buildConfig(interBrokerProtocol: MetadataVersion, messageFormat: MetadataVersion): KafkaConfig = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.InterBrokerProtocolVersionProp, interBrokerProtocol.version) - props.put(KafkaConfig.LogMessageFormatVersionProp, messageFormat.version) + props.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, interBrokerProtocol.version) + props.setProperty(KafkaConfig.LogMessageFormatVersionProp, messageFormat.version) KafkaConfig.fromProps(props) } @@ -718,7 +718,7 @@ class KafkaConfigTest { def testFromPropsInvalid(): Unit = { def baseProperties: Properties = { val validRequiredProperties = new Properties() - validRequiredProperties.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") + validRequiredProperties.setProperty(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") validRequiredProperties } // to ensure a basis is valid - bootstraps all needed validation @@ -946,6 +946,8 @@ class KafkaConfigTest { case KafkaConfig.KafkaMetricsReporterClassesProp => // ignore case KafkaConfig.KafkaMetricsPollingIntervalSecondsProp => //ignore + case KafkaConfig.SaslServerMaxReceiveSizeProp => assertPropertyInvalid(baseProperties, name, "not_a_number") + // Raft Quorum Configs case RaftConfig.QUORUM_VOTERS_CONFIG => // ignore string case RaftConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") @@ -986,7 +988,7 @@ class KafkaConfigTest { def testDynamicLogConfigs(): Unit = { def baseProperties: Properties = { val validRequiredProperties = new Properties() - validRequiredProperties.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") + validRequiredProperties.setProperty(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") validRequiredProperties } @@ -995,7 +997,7 @@ class KafkaConfigTest { def assertDynamic(property: String, value: Any, accessor: () => Any): Unit = { val initial = accessor() - props.put(property, value.toString) + props.setProperty(property, value.toString) config.updateCurrentConfig(new KafkaConfig(props)) assertNotEquals(initial, accessor()) } @@ -1067,23 +1069,23 @@ class KafkaConfigTest { @Test def testSpecificProperties(): Unit = { val defaults = new Properties() - defaults.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") + defaults.setProperty(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") // For ZkConnectionTimeoutMs - defaults.put(KafkaConfig.ZkSessionTimeoutMsProp, "1234") - defaults.put(KafkaConfig.BrokerIdGenerationEnableProp, "false") - defaults.put(KafkaConfig.MaxReservedBrokerIdProp, "1") - defaults.put(KafkaConfig.BrokerIdProp, "1") - defaults.put(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:1122") - defaults.put(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.1:2, 127.0.0.2:3") - defaults.put(KafkaConfig.LogDirProp, "/tmp1,/tmp2") - defaults.put(KafkaConfig.LogRollTimeHoursProp, "12") - defaults.put(KafkaConfig.LogRollTimeJitterHoursProp, "11") - defaults.put(KafkaConfig.LogRetentionTimeHoursProp, "10") + defaults.setProperty(KafkaConfig.ZkSessionTimeoutMsProp, "1234") + defaults.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") + defaults.setProperty(KafkaConfig.MaxReservedBrokerIdProp, "1") + defaults.setProperty(KafkaConfig.BrokerIdProp, "1") + defaults.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:1122") + defaults.setProperty(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.1:2, 127.0.0.2:3") + defaults.setProperty(KafkaConfig.LogDirProp, "/tmp1,/tmp2") + defaults.setProperty(KafkaConfig.LogRollTimeHoursProp, "12") + defaults.setProperty(KafkaConfig.LogRollTimeJitterHoursProp, "11") + defaults.setProperty(KafkaConfig.LogRetentionTimeHoursProp, "10") //For LogFlushIntervalMsProp - defaults.put(KafkaConfig.LogFlushSchedulerIntervalMsProp, "123") - defaults.put(KafkaConfig.OffsetsTopicCompressionCodecProp, SnappyCompressionCodec.codec.toString) + defaults.setProperty(KafkaConfig.LogFlushSchedulerIntervalMsProp, "123") + defaults.setProperty(KafkaConfig.OffsetsTopicCompressionCodecProp, SnappyCompressionCodec.codec.toString) // For MetricRecordingLevelProp - defaults.put(KafkaConfig.MetricRecordingLevelProp, Sensor.RecordingLevel.DEBUG.toString) + defaults.setProperty(KafkaConfig.MetricRecordingLevelProp, Sensor.RecordingLevel.DEBUG.toString) val config = KafkaConfig.fromProps(defaults) assertEquals("127.0.0.1:2181", config.zkConnect) @@ -1105,7 +1107,7 @@ class KafkaConfigTest { assertEquals(24 * 60L * 60L * 1000L, config.delegationTokenExpiryTimeMs) assertEquals(1 * 60L * 1000L * 60, config.delegationTokenExpiryCheckIntervalMs) - defaults.put(KafkaConfig.DelegationTokenSecretKeyProp, "1234567890") + defaults.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "1234567890") val config1 = KafkaConfig.fromProps(defaults) assertEquals(true, config1.tokenAuthEnabled) } @@ -1113,19 +1115,19 @@ class KafkaConfigTest { @Test def testNonroutableAdvertisedListeners(): Unit = { val props = new Properties() - props.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://0.0.0.0:9092") + props.setProperty(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://0.0.0.0:9092") assertFalse(isValidKafkaConfig(props)) } @Test def testMaxConnectionsPerIpProp(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.MaxConnectionsPerIpProp, "0") + props.setProperty(KafkaConfig.MaxConnectionsPerIpProp, "0") assertFalse(isValidKafkaConfig(props)) - props.put(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.1:100") + props.setProperty(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.1:100") KafkaConfig.fromProps(props) - props.put(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.0#:100") + props.setProperty(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.0#:100") assertFalse(isValidKafkaConfig(props)) } @@ -1143,19 +1145,19 @@ class KafkaConfigTest { @Test def testDistinctControllerAndAdvertisedListenersAllowedForKRaftBroker(): Unit = { val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "broker") - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094") - props.put(KafkaConfig.AdvertisedListenersProp, "PLAINTEXT://A:9092,SSL://B:9093") // explicitly setting it in KRaft - props.put(KafkaConfig.ControllerListenerNamesProp, "SASL_SSL") - props.put(KafkaConfig.NodeIdProp, "2") - props.put(KafkaConfig.QuorumVotersProp, "3@localhost:9094") + props.setProperty(KafkaConfig.ProcessRolesProp, "broker") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094") + props.setProperty(KafkaConfig.AdvertisedListenersProp, "PLAINTEXT://A:9092,SSL://B:9093") // explicitly setting it in KRaft + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SASL_SSL") + props.setProperty(KafkaConfig.NodeIdProp, "2") + props.setProperty(KafkaConfig.QuorumVotersProp, "3@localhost:9094") // invalid due to extra listener also appearing in controller listeners assertBadConfigContainingMessage(props, "controller.listener.names must not contain a value appearing in the 'listeners' configuration when running KRaft with just the broker role") // Valid now - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://A:9092,SSL://B:9093") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://A:9092,SSL://B:9093") KafkaConfig.fromProps(props) // Also valid if we let advertised listeners be derived from listeners/controller.listener.names @@ -1167,19 +1169,19 @@ class KafkaConfigTest { @Test def testControllerListenersCannotBeAdvertisedForKRaftBroker(): Unit = { val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "broker,controller") + props.setProperty(KafkaConfig.ProcessRolesProp, "broker,controller") val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" - props.put(KafkaConfig.ListenersProp, listeners) - props.put(KafkaConfig.AdvertisedListenersProp, listeners) // explicitly setting it in KRaft - props.put(KafkaConfig.InterBrokerListenerNameProp, "SASL_SSL") - props.put(KafkaConfig.ControllerListenerNamesProp, "PLAINTEXT,SSL") - props.put(KafkaConfig.NodeIdProp, "2") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9092") + props.setProperty(KafkaConfig.ListenersProp, listeners) + props.setProperty(KafkaConfig.AdvertisedListenersProp, listeners) // explicitly setting it in KRaft + props.setProperty(KafkaConfig.InterBrokerListenerNameProp, "SASL_SSL") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "PLAINTEXT,SSL") + props.setProperty(KafkaConfig.NodeIdProp, "2") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9092") assertBadConfigContainingMessage(props, "The advertised.listeners config must not contain KRaft controller listeners from controller.listener.names when process.roles contains the broker role") // Valid now - props.put(KafkaConfig.AdvertisedListenersProp, "SASL_SSL://C:9094") + props.setProperty(KafkaConfig.AdvertisedListenersProp, "SASL_SSL://C:9094") KafkaConfig.fromProps(props) // Also valid if we allow advertised listeners to derive from listeners/controller.listener.names @@ -1189,28 +1191,44 @@ class KafkaConfigTest { @Test def testAdvertisedListenersDisallowedForKRaftControllerOnlyRole(): Unit = { - val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "controller") - val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" - props.put(KafkaConfig.ListenersProp, listeners) - props.put(KafkaConfig.AdvertisedListenersProp, listeners) // explicitly setting it in KRaft - props.put(KafkaConfig.ControllerListenerNamesProp, "PLAINTEXT,SSL") - props.put(KafkaConfig.NodeIdProp, "2") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9092") - val expectedExceptionContainsTextSuffix = " config must only contain KRaft controller listeners from controller.listener.names when process.roles=controller" - assertBadConfigContainingMessage(props, "The advertised.listeners" + expectedExceptionContainsTextSuffix) - - // Still invalid due to extra listener if we set advertised listeners explicitly to be correct + // Test that advertised listeners cannot be set when KRaft and server is controller only. + // Test that listeners must enumerate every controller listener + // Test that controller listener must enumerate every listener val correctListeners = "PLAINTEXT://A:9092,SSL://B:9093" - props.put(KafkaConfig.AdvertisedListenersProp, correctListeners) - assertBadConfigContainingMessage(props, "The advertised.listeners" + expectedExceptionContainsTextSuffix) + val incorrectListeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" - // Still invalid due to extra listener if we allow advertised listeners to derive from listeners/controller.listener.names + val correctControllerListenerNames = "PLAINTEXT,SSL" + + val props = new Properties() + props.setProperty(KafkaConfig.ProcessRolesProp, "controller") + props.setProperty(KafkaConfig.ListenersProp, correctListeners) + props.setProperty(KafkaConfig.AdvertisedListenersProp, incorrectListeners) + props.setProperty(KafkaConfig.ControllerListenerNamesProp, correctControllerListenerNames) + props.setProperty(KafkaConfig.NodeIdProp, "2") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9092") + var expectedExceptionContainsText = "The advertised.listeners config must be empty when process.roles=controller" + assertBadConfigContainingMessage(props, expectedExceptionContainsText) + + // Invalid if advertised listeners is explicitly to the set + props.setProperty(KafkaConfig.AdvertisedListenersProp, correctListeners) + assertBadConfigContainingMessage(props, expectedExceptionContainsText) + + // Invalid if listeners contains names not in controller.listener.names props.remove(KafkaConfig.AdvertisedListenersProp) - assertBadConfigContainingMessage(props, "The listeners" + expectedExceptionContainsTextSuffix) + props.setProperty(KafkaConfig.ListenersProp, incorrectListeners) + expectedExceptionContainsText = """The listeners config must only contain KRaft controller listeners from + |controller.listener.names when process.roles=controller""".stripMargin.replaceAll("\n", " ") + assertBadConfigContainingMessage(props, expectedExceptionContainsText) + + // Invalid if listeners doesn't contain every name in controller.listener.names + props.setProperty(KafkaConfig.ListenersProp, correctListeners) + props.setProperty(KafkaConfig.ControllerListenerNamesProp, correctControllerListenerNames + ",SASL_SSL") + expectedExceptionContainsText = """controller.listener.names must only contain values appearing in the 'listeners' + |configuration when running the KRaft controller role""".stripMargin.replaceAll("\n", " ") + assertBadConfigContainingMessage(props, expectedExceptionContainsText) // Valid now - props.put(KafkaConfig.ListenersProp, correctListeners) + props.setProperty(KafkaConfig.ControllerListenerNamesProp, correctControllerListenerNames) KafkaConfig.fromProps(props) } @@ -1242,7 +1260,7 @@ class KafkaConfigTest { private def assertInvalidQuorumVoters(value: String): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) - props.put(KafkaConfig.QuorumVotersProp, value) + props.setProperty(KafkaConfig.QuorumVotersProp, value) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @@ -1267,7 +1285,7 @@ class KafkaConfigTest { private def assertValidQuorumVoters(value: String, expectedVoters: util.Map[Integer, AddressSpec]): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) - props.put(KafkaConfig.QuorumVotersProp, value) + props.setProperty(KafkaConfig.QuorumVotersProp, value) val raftConfig = new RaftConfig(KafkaConfig.fromProps(props)) assertEquals(expectedVoters, raftConfig.quorumVoterConnections()) } @@ -1279,11 +1297,11 @@ class KafkaConfigTest { // and make sure it is allowed despite broker.id.generation.enable=true (true is the default) val largeBrokerId = 2000 val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "broker") - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092") - props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9093") - props.put(KafkaConfig.NodeIdProp, largeBrokerId.toString) + props.setProperty(KafkaConfig.ProcessRolesProp, "broker") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://localhost:9092") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") + props.setProperty(KafkaConfig.NodeIdProp, largeBrokerId.toString) KafkaConfig.fromProps(props) } @@ -1291,7 +1309,7 @@ class KafkaConfigTest { def testRejectsNegativeNodeIdForRaftBasedBrokerCaseWithAutoGenEnabled(): Unit = { // -1 is the default for both node.id and broker.id val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "broker") + props.setProperty(KafkaConfig.ProcessRolesProp, "broker") assertFalse(isValidKafkaConfig(props)) } @@ -1299,7 +1317,7 @@ class KafkaConfigTest { def testRejectsNegativeNodeIdForRaftBasedControllerCaseWithAutoGenEnabled(): Unit = { // -1 is the default for both node.id and broker.id val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "controller") + props.setProperty(KafkaConfig.ProcessRolesProp, "controller") assertFalse(isValidKafkaConfig(props)) } @@ -1307,9 +1325,9 @@ class KafkaConfigTest { def testRejectsNegativeNodeIdForRaftBasedCaseWithAutoGenDisabled(): Unit = { // -1 is the default for both node.id and broker.id val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "broker") - props.put(KafkaConfig.BrokerIdGenerationEnableProp, "false") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9093") + props.setProperty(KafkaConfig.ProcessRolesProp, "broker") + props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") assertFalse(isValidKafkaConfig(props)) } @@ -1321,8 +1339,8 @@ class KafkaConfigTest { val largeBrokerId = 2000 val props = TestUtils.createBrokerConfig(largeBrokerId, TestUtils.MockZkConnect, port = TestUtils.MockZkPort) val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" - props.put(KafkaConfig.ListenersProp, listeners) - props.put(KafkaConfig.AdvertisedListenersProp, listeners) + props.setProperty(KafkaConfig.ListenersProp, listeners) + props.setProperty(KafkaConfig.AdvertisedListenersProp, listeners) assertFalse(isValidKafkaConfig(props)) } @@ -1331,8 +1349,8 @@ class KafkaConfigTest { // -1 is the default for both node.id and broker.id; it implies "auto-generate" and should succeed val props = TestUtils.createBrokerConfig(-1, TestUtils.MockZkConnect, port = TestUtils.MockZkPort) val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" - props.put(KafkaConfig.ListenersProp, listeners) - props.put(KafkaConfig.AdvertisedListenersProp, listeners) + props.setProperty(KafkaConfig.ListenersProp, listeners) + props.setProperty(KafkaConfig.AdvertisedListenersProp, listeners) KafkaConfig.fromProps(props) } @@ -1342,10 +1360,10 @@ class KafkaConfigTest { val negativeTwoNodeId = -2 val props = TestUtils.createBrokerConfig(negativeTwoNodeId, TestUtils.MockZkConnect, port = TestUtils.MockZkPort) val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" - props.put(KafkaConfig.ListenersProp, listeners) - props.put(KafkaConfig.AdvertisedListenersProp, listeners) - props.put(KafkaConfig.NodeIdProp, negativeTwoNodeId.toString) - props.put(KafkaConfig.BrokerIdProp, negativeTwoNodeId.toString) + props.setProperty(KafkaConfig.ListenersProp, listeners) + props.setProperty(KafkaConfig.AdvertisedListenersProp, listeners) + props.setProperty(KafkaConfig.NodeIdProp, negativeTwoNodeId.toString) + props.setProperty(KafkaConfig.BrokerIdProp, negativeTwoNodeId.toString) assertFalse(isValidKafkaConfig(props)) } @@ -1356,9 +1374,9 @@ class KafkaConfigTest { val largeBrokerId = 2000 val props = TestUtils.createBrokerConfig(largeBrokerId, TestUtils.MockZkConnect, port = TestUtils.MockZkPort) val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" - props.put(KafkaConfig.ListenersProp, listeners) - props.put(KafkaConfig.AdvertisedListenersProp, listeners) - props.put(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props.setProperty(KafkaConfig.ListenersProp, listeners) + props.setProperty(KafkaConfig.AdvertisedListenersProp, listeners) + props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") KafkaConfig.fromProps(props) } @@ -1367,27 +1385,27 @@ class KafkaConfigTest { // -1 is the default for both node.id and broker.id val props = TestUtils.createBrokerConfig(-1, TestUtils.MockZkConnect, port = TestUtils.MockZkPort) val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" - props.put(KafkaConfig.ListenersProp, listeners) - props.put(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props.setProperty(KafkaConfig.ListenersProp, listeners) + props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") assertFalse(isValidKafkaConfig(props)) } @Test def testZookeeperConnectRequiredIfEmptyProcessRoles(): Unit = { val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "") - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:9092") + props.setProperty(KafkaConfig.ProcessRolesProp, "") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:9092") assertFalse(isValidKafkaConfig(props)) } @Test def testZookeeperConnectNotRequiredIfNonEmptyProcessRoles(): Unit = { val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "broker") - props.put(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:9092") - props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") - props.put(KafkaConfig.NodeIdProp, "1") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9093") + props.setProperty(KafkaConfig.ProcessRolesProp, "broker") + props.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:9092") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") + props.setProperty(KafkaConfig.NodeIdProp, "1") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") KafkaConfig.fromProps(props) } @@ -1397,12 +1415,12 @@ class KafkaConfigTest { val dataDir = "/path/to/data/dir" val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "broker") - props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") - props.put(KafkaConfig.MetadataLogDirProp, metadataDir) - props.put(KafkaConfig.LogDirProp, dataDir) - props.put(KafkaConfig.NodeIdProp, "1") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9093") + props.setProperty(KafkaConfig.ProcessRolesProp, "broker") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") + props.setProperty(KafkaConfig.MetadataLogDirProp, metadataDir) + props.setProperty(KafkaConfig.LogDirProp, dataDir) + props.setProperty(KafkaConfig.NodeIdProp, "1") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") KafkaConfig.fromProps(props) val config = KafkaConfig.fromProps(props) @@ -1416,11 +1434,11 @@ class KafkaConfigTest { val dataDir2 = "/path/to/data/dir/2" val props = new Properties() - props.put(KafkaConfig.ProcessRolesProp, "broker") - props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") - props.put(KafkaConfig.LogDirProp, s"$dataDir1,$dataDir2") - props.put(KafkaConfig.NodeIdProp, "1") - props.put(KafkaConfig.QuorumVotersProp, "2@localhost:9093") + props.setProperty(KafkaConfig.ProcessRolesProp, "broker") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") + props.setProperty(KafkaConfig.LogDirProp, s"$dataDir1,$dataDir2") + props.setProperty(KafkaConfig.NodeIdProp, "1") + props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") KafkaConfig.fromProps(props) val config = KafkaConfig.fromProps(props) @@ -1476,7 +1494,7 @@ class KafkaConfigTest { def testNodeIdIsInferredByBrokerIdWithKraft(): Unit = { val props = new Properties() props.setProperty(KafkaConfig.ProcessRolesProp, "broker") - props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") + props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KafkaConfig.BrokerIdProp, "3") props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") val config = KafkaConfig.fromProps(props) @@ -1498,7 +1516,7 @@ class KafkaConfigTest { @Test def testBrokerIdIsInferredByNodeIdWithKraft(): Unit = { - val props = new Properties(kraftProps()) + val props = new Properties() props.putAll(kraftProps()) val config = KafkaConfig.fromProps(props) assertEquals(3, config.brokerId) @@ -1511,7 +1529,7 @@ class KafkaConfigTest { @Test def testSaslJwksEndpointRetryDefaults(): Unit = { val props = new Properties() - props.put(KafkaConfig.ZkConnectProp, "localhost:2181") + props.setProperty(KafkaConfig.ZkConnectProp, "localhost:2181") val config = KafkaConfig.fromProps(props) assertNotNull(config.getLong(KafkaConfig.SaslOAuthBearerJwksEndpointRetryBackoffMsProp)) assertNotNull(config.getLong(KafkaConfig.SaslOAuthBearerJwksEndpointRetryBackoffMaxMsProp)) @@ -1529,7 +1547,7 @@ class KafkaConfigTest { @Test def testInvalidSecurityInterBrokerProtocol(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaConfig.InterBrokerSecurityProtocolProp, "abc") + props.setProperty(KafkaConfig.InterBrokerSecurityProtocolProp, "abc") val ce = assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) assertTrue(ce.getMessage.contains(KafkaConfig.InterBrokerSecurityProtocolProp)) } @@ -1595,7 +1613,7 @@ class KafkaConfigTest { def testDefaultInterBrokerProtocolVersionKRaft(): Unit = { val props = new Properties() props.putAll(kraftProps()) - val config = new KafkaConfig(props) + val config = KafkaConfig.fromProps(props) assertEquals(config.interBrokerProtocolVersion, MetadataVersion.MINIMUM_KRAFT_VERSION) } } diff --git a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala index 7e5d791db252b..1adf544819fa3 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala @@ -17,15 +17,13 @@ package kafka.server import java.util - import java.util.concurrent.atomic.AtomicReference - -import kafka.utils.{CoreUtils, TestUtils} -import kafka.server.QuorumTestHarness +import kafka.utils.{CoreUtils, TestInfoUtils, TestUtils} import org.apache.kafka.common.metrics.{KafkaMetric, MetricsContext, MetricsReporter} -import org.junit.jupiter.api.Assertions.{assertEquals} -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource object KafkaMetricsReporterTest { @@ -43,52 +41,63 @@ object KafkaMetricsReporterTest { override def contextChange(metricsContext: MetricsContext): Unit = { //read jmxPrefix - MockMetricsReporter.JMXPREFIX.set(metricsContext.contextLabels().get("_namespace").toString) - MockMetricsReporter.CLUSTERID.set(metricsContext.contextLabels().get("kafka.cluster.id").toString) - MockMetricsReporter.BROKERID.set(metricsContext.contextLabels().get("kafka.broker.id").toString) + MockMetricsReporter.JMXPREFIX.set(contextLabelOrNull("_namespace", metricsContext)) + MockMetricsReporter.CLUSTERID.set(contextLabelOrNull("kafka.cluster.id", metricsContext)) + MockMetricsReporter.BROKERID.set(contextLabelOrNull("kafka.broker.id", metricsContext)) + MockMetricsReporter.NODEID.set(contextLabelOrNull("kafka.node.id", metricsContext)) } - override def configure(configs: util.Map[String, _]): Unit = {} + private def contextLabelOrNull(name: String, metricsContext: MetricsContext): String = { + Option(metricsContext.contextLabels().get(name)).flatMap(v => Option(v.toString())).getOrElse(null) + } + override def configure(configs: util.Map[String, _]): Unit = {} } object MockMetricsReporter { val JMXPREFIX: AtomicReference[String] = new AtomicReference[String] val BROKERID : AtomicReference[String] = new AtomicReference[String] + val NODEID : AtomicReference[String] = new AtomicReference[String] val CLUSTERID : AtomicReference[String] = new AtomicReference[String] } } class KafkaMetricsReporterTest extends QuorumTestHarness { - var server: KafkaServer = null + var broker: KafkaBroker = null var config: KafkaConfig = null @BeforeEach override def setUp(testInfo: TestInfo): Unit = { super.setUp(testInfo) - val props = TestUtils.createBrokerConfig(1, zkConnect) + val props = TestUtils.createBrokerConfig(1, zkConnectOrNull) props.setProperty(KafkaConfig.MetricReporterClassesProp, "kafka.server.KafkaMetricsReporterTest$MockMetricsReporter") props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "true") - props.setProperty(KafkaConfig.BrokerIdProp, "-1") + props.setProperty(KafkaConfig.BrokerIdProp, "1") config = KafkaConfig.fromProps(props) - server = new KafkaServer(config, threadNamePrefix = Option(this.getClass.getName)) - server.startup() + broker = createBroker(config, threadNamePrefix = Option(this.getClass.getName)) + broker.startup() } - @Test - def testMetricsContextNamespacePresent(): Unit = { - assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.CLUSTERID) - assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.BROKERID) - assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.JMXPREFIX) - assertEquals("kafka.server", KafkaMetricsReporterTest.MockMetricsReporter.JMXPREFIX.get()) + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + def testMetricsContextNamespacePresent(quorum: String): Unit = { + assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.CLUSTERID.get()) + if (isKRaftTest()) { + assertNull(KafkaMetricsReporterTest.MockMetricsReporter.BROKERID.get()) + assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.NODEID.get()) + } else { + assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.BROKERID.get()) + assertNull(KafkaMetricsReporterTest.MockMetricsReporter.NODEID.get()) + } + assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.JMXPREFIX.get()) - server.shutdown() + broker.shutdown() TestUtils.assertNoNonDaemonThreads(this.getClass.getName) } @AfterEach override def tearDown(): Unit = { - server.shutdown() + broker.shutdown() CoreUtils.delete(config.logDirs) super.tearDown() } diff --git a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala index 17483e58a6a05..1fc98fac3ea61 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala @@ -18,12 +18,12 @@ package kafka.server import java.io.File import java.nio.file.Files -import java.util.Properties +import java.util.{Optional, Properties} import kafka.common.{InconsistentBrokerMetadataException, InconsistentNodeIdException} import kafka.log.UnifiedLog import org.apache.kafka.common.{KafkaException, Uuid} import org.apache.kafka.common.utils.Utils -import org.apache.kafka.controller.BootstrapMetadata +import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.test.TestUtils import org.junit.jupiter.api.Assertions._ @@ -97,8 +97,8 @@ class KafkaRaftServerTest { } private def writeBootstrapMetadata(logDir: File, metadataVersion: MetadataVersion): Unit = { - val bootstrapMetadata = BootstrapMetadata.create(metadataVersion) - BootstrapMetadata.write(bootstrapMetadata, logDir.toPath) + val bootstrapDirectory = new BootstrapDirectory(logDir.toString(), Optional.empty()) + bootstrapDirectory.writeBinaryFile(BootstrapMetadata.fromVersion(metadataVersion, "test")) } @Test @@ -235,14 +235,14 @@ class KafkaRaftServerTest { configProperties.put(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093") configProperties.put(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093") configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") - configProperties.put(KafkaConfig.InterBrokerProtocolVersionProp, "3.2") + configProperties.put(KafkaConfig.InterBrokerProtocolVersionProp, "3.3-IV1") val (loadedMetaProperties, bootstrapMetadata, offlineDirs) = invokeLoadMetaProperties(metaProperties, configProperties, None) assertEquals(metaProperties, loadedMetaProperties) assertEquals(Seq.empty, offlineDirs) - assertEquals(bootstrapMetadata.metadataVersion(), MetadataVersion.IBP_3_2_IV0) + assertEquals(bootstrapMetadata.metadataVersion(), MetadataVersion.IBP_3_3_IV1) } @Test @@ -262,8 +262,11 @@ class KafkaRaftServerTest { configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KafkaConfig.LogDirProp, logDir.getAbsolutePath) - val config = KafkaConfig.fromProps(configProperties) - assertEquals("Cannot upgrade from KRaft version prior to 3.3 without first setting inter.broker.protocol.version on each broker.", - assertThrows(classOf[KafkaException], () => KafkaRaftServer.initializeLogDirs(config)).getMessage) + val (loadedMetaProperties, bootstrapMetadata, offlineDirs) = + invokeLoadMetaProperties(metaProperties, configProperties, None) + + assertEquals(metaProperties, loadedMetaProperties) + assertEquals(Seq.empty, offlineDirs) + assertEquals(bootstrapMetadata.metadataVersion(), MetadataVersion.latest()) } } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala index c7a222c2d547d..8eeb22046e6ee 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala @@ -34,18 +34,20 @@ import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRec import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET} import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, UpdateMetadataRequest} import org.apache.kafka.common.utils.{LogContext, SystemTime} +import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0 import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, anyBoolean, anyLong} import org.mockito.Mockito.{mock, never, times, verify, when} + import java.nio.charset.StandardCharsets import java.util import java.util.{Collections, Optional} -import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0 - import scala.collection.{Map, mutable} import scala.jdk.CollectionConverters._ @@ -1100,6 +1102,75 @@ class ReplicaFetcherThreadTest { assertEquals(Collections.singletonList(tid1p0), fetchRequestBuilder2.removed()) } + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testLocalFetchCompletionIfHighWatermarkUpdated(highWatermarkUpdated: Boolean): Unit = { + val props = TestUtils.createBrokerConfig(1, "localhost:1234") + val config = KafkaConfig.fromProps(props) + val highWatermarkReceivedFromLeader = 100L + + val mockBlockingSend: BlockingSend = mock(classOf[BlockingSend]) + when(mockBlockingSend.brokerEndPoint()).thenReturn(brokerEndPoint) + + val maybeNewHighWatermark = if (highWatermarkUpdated) { + Some(highWatermarkReceivedFromLeader) + } else { + None + } + val log: UnifiedLog = mock(classOf[UnifiedLog]) + when(log.maybeUpdateHighWatermark(highWatermarkReceivedFromLeader)) + .thenReturn(maybeNewHighWatermark) + + val appendInfo: Option[LogAppendInfo] = Some(mock(classOf[LogAppendInfo])) + + val partition: Partition = mock(classOf[Partition]) + when(partition.localLogOrException).thenReturn(log) + when(partition.appendRecordsToFollowerOrFutureReplica(any[MemoryRecords], any[Boolean])).thenReturn(appendInfo) + + // In Scala 2.12, the partitionsWithNewHighWatermark buffer is cleared before the replicaManager mock is verified. + // Capture the argument at the time of invocation. + val completeDelayedFetchRequestsArgument = mutable.Buffer.empty[TopicPartition] + val replicaManager: ReplicaManager = mock(classOf[ReplicaManager]) + when(replicaManager.getPartitionOrException(any[TopicPartition])).thenReturn(partition) + when(replicaManager.completeDelayedFetchRequests(any[Seq[TopicPartition]])).thenAnswer(invocation => + completeDelayedFetchRequestsArgument ++= invocation.getArguments()(0).asInstanceOf[Seq[TopicPartition]] + ) + val brokerTopicStats = new BrokerTopicStats + when(replicaManager.brokerTopicStats).thenReturn(brokerTopicStats) + + val replicaQuota: ReplicaQuota = mock(classOf[ReplicaQuota]) + + val thread = createReplicaFetcherThread( + name = "replica-fetcher", + fetcherId = 0, + brokerConfig = config, + failedPartitions = failedPartitions, + replicaMgr = replicaManager, + quota = replicaQuota, + leaderEndpointBlockingSend = mockBlockingSend) + + val tp0 = new TopicPartition("testTopic", 0) + val tp1 = new TopicPartition("testTopic", 1) + val records = MemoryRecords.withRecords(CompressionType.NONE, + new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8))) + val partitionData = new FetchResponseData.PartitionData() + .setRecords(records) + .setHighWatermark(highWatermarkReceivedFromLeader) + + thread.processPartitionData(tp0, 0, partitionData.setPartitionIndex(0)) + thread.processPartitionData(tp1, 0, partitionData.setPartitionIndex(1)) + verify(replicaManager, times(0)).completeDelayedFetchRequests(any[Seq[TopicPartition]]) + + thread.doWork() + if (highWatermarkUpdated) { + assertEquals(Seq(tp0, tp1), completeDelayedFetchRequestsArgument) + verify(replicaManager, times(1)).completeDelayedFetchRequests(any[Seq[TopicPartition]]) + } else { + verify(replicaManager, times(0)).completeDelayedFetchRequests(any[Seq[TopicPartition]]) + } + assertEquals(mutable.Buffer.empty, thread.partitionsWithNewHighWatermark) + } + private def newOffsetForLeaderPartitionResult( tp: TopicPartition, leaderEpoch: Int, @@ -1129,11 +1200,15 @@ class ReplicaFetcherThreadTest { when(mockBlockingSend.brokerEndPoint()).thenReturn(brokerEndPoint) val log: UnifiedLog = mock(classOf[UnifiedLog]) + val records = MemoryRecords.withRecords(CompressionType.NONE, + new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8))) + when(log.maybeUpdateHighWatermark(hw = 0)).thenReturn(None) val partition: Partition = mock(classOf[Partition]) when(partition.localLogOrException).thenReturn(log) when(partition.isReassigning).thenReturn(isReassigning) when(partition.isAddingLocalReplica).thenReturn(isReassigning) + when(partition.appendRecordsToFollowerOrFutureReplica(records, isFuture = false)).thenReturn(None) val replicaManager: ReplicaManager = mock(classOf[ReplicaManager]) when(replicaManager.getPartitionOrException(any[TopicPartition])).thenReturn(partition) @@ -1152,8 +1227,6 @@ class ReplicaFetcherThreadTest { mockBlockingSend ) - val records = MemoryRecords.withRecords(CompressionType.NONE, - new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8))) val partitionData: thread.FetchData = new FetchResponseData.PartitionData() .setPartitionIndex(t1p0.partition) .setLastStableOffset(0) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 8050a36d1d942..d6330eaf78d32 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -21,7 +21,7 @@ import java.io.File import java.net.InetAddress import java.nio.file.Files import java.util -import java.util.concurrent.atomic.{AtomicLong, AtomicReference} +import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong, AtomicReference} import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.stream.IntStream import java.util.{Collections, Optional, Properties} @@ -47,6 +47,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record._ import org.apache.kafka.common.replica.{ClientMetadata, PartitionView, ReplicaSelector, ReplicaView} import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata +import org.apache.kafka.common.replica.ReplicaView.DefaultReplicaView import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests._ @@ -1189,7 +1190,7 @@ class ReplicaManagerTest { val tidp0 = new TopicIdPartition(topicId, tp0) // Make this replica the follower - val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, + val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, Seq(new LeaderAndIsrPartitionState() .setTopicName(topic) .setPartitionIndex(0) @@ -1202,7 +1203,7 @@ class ReplicaManagerTest { .setIsNew(false)).asJava, Collections.singletonMap(topic, topicId), Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() - replicaManager.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ()) + replicaManager.becomeLeaderOrFollower(1, leaderAndIsrRequest, (_, _) => ()) val metadata: ClientMetadata = new DefaultClientMetadata("rack-a", "client-id", InetAddress.getByName("localhost"), KafkaPrincipal.ANONYMOUS, "default") @@ -1245,7 +1246,7 @@ class ReplicaManagerTest { val tidp0 = new TopicIdPartition(topicId, tp0) // Make this replica the leader - val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, + val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, Seq(new LeaderAndIsrPartitionState() .setTopicName(topic) .setPartitionIndex(0) @@ -1258,7 +1259,7 @@ class ReplicaManagerTest { .setIsNew(false)).asJava, Collections.singletonMap(topic, topicId), Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build() - replicaManager.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ()) + replicaManager.becomeLeaderOrFollower(1, leaderAndIsrRequest, (_, _) => ()) val metadata = new DefaultClientMetadata("rack-a", "client-id", InetAddress.getByName("localhost"), KafkaPrincipal.ANONYMOUS, "default") @@ -1279,6 +1280,86 @@ class ReplicaManagerTest { TestUtils.assertNoNonDaemonThreads(this.getClass.getName) } + @Test + def testPreferredReplicaAsLeaderWhenSameRackFollowerIsOutOfIsr(): Unit = { + val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), + propsModifier = props => props.put(KafkaConfig.ReplicaSelectorClassProp, classOf[MockReplicaSelector].getName)) + + try { + val leaderBrokerId = 0 + val followerBrokerId = 1 + val leaderNode = new Node(leaderBrokerId, "host1", 0, "rack-a") + val followerNode = new Node(followerBrokerId, "host2", 1, "rack-b") + val brokerList = Seq[Integer](leaderBrokerId, followerBrokerId).asJava + val topicId = Uuid.randomUuid() + val tp0 = new TopicPartition(topic, 0) + val tidp0 = new TopicIdPartition(topicId, tp0) + + when(replicaManager.metadataCache.getPartitionReplicaEndpoints( + tp0, + new ListenerName("default") + )).thenReturn(Map( + leaderBrokerId -> leaderNode, + followerBrokerId -> followerNode + ).toMap) + + // Make this replica the leader and remove follower from ISR. + val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder( + ApiKeys.LEADER_AND_ISR.latestVersion, + 0, + 0, + brokerEpoch, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(leaderBrokerId) + .setLeaderEpoch(1) + .setIsr(Seq[Integer](leaderBrokerId).asJava) + .setPartitionEpoch(0) + .setReplicas(brokerList) + .setIsNew(false)).asJava, + Collections.singletonMap(topic, topicId), + Set(leaderNode, followerNode).asJava).build() + + replicaManager.becomeLeaderOrFollower(2, leaderAndIsrRequest, (_, _) => ()) + + appendRecords(replicaManager, tp0, TestUtils.singletonRecords(s"message".getBytes)).onFire { response => + assertEquals(Errors.NONE, response.error) + } + // Fetch as follower to initialise the log end offset of the replica + fetchPartitionAsFollower( + replicaManager, + new TopicIdPartition(topicId, new TopicPartition(topic, 0)), + new PartitionData(Uuid.ZERO_UUID, 0, 0, 100000, Optional.empty()), + replicaId = 1 + ) + + val metadata = new DefaultClientMetadata("rack-b", "client-id", + InetAddress.getByName("localhost"), KafkaPrincipal.ANONYMOUS, "default") + + val consumerResult = fetchPartitionAsConsumer( + replicaManager, + tidp0, + new PartitionData(Uuid.ZERO_UUID, 0, 0, 100000, Optional.empty()), + clientMetadata = Some(metadata) + ) + + // Fetch from leader succeeds + assertTrue(consumerResult.hasFired) + + // PartitionView passed to ReplicaSelector should not contain the follower as it's not in the ISR + val expectedReplicaViews = Set(new DefaultReplicaView(leaderNode, 1, 0)) + val partitionView = replicaManager.replicaSelectorOpt.get + .asInstanceOf[MockReplicaSelector].getPartitionViewArgument + + assertTrue(partitionView.isDefined) + assertEquals(expectedReplicaViews.asJava, partitionView.get.replicas) + } finally { + replicaManager.shutdown() + } + } + @Test def testFetchFromFollowerShouldNotRunPreferLeaderSelect(): Unit = { val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), @@ -2210,7 +2291,8 @@ class ReplicaManagerTest { aliveBrokerIds: Seq[Int] = Seq(0, 1), propsModifier: Properties => Unit = _ => {}, mockReplicaFetcherManager: Option[ReplicaFetcherManager] = None, - mockReplicaAlterLogDirsManager: Option[ReplicaAlterLogDirsManager] = None + mockReplicaAlterLogDirsManager: Option[ReplicaAlterLogDirsManager] = None, + isShuttingDown: AtomicBoolean = new AtomicBoolean(false) ): ReplicaManager = { val props = TestUtils.createBrokerConfig(brokerId, TestUtils.MockZkConnect) props.put("log.dirs", TestUtils.tempRelativeDir("data").getAbsolutePath + "," + TestUtils.tempRelativeDir("data2").getAbsolutePath) @@ -2245,6 +2327,7 @@ class ReplicaManagerTest { metadataCache = metadataCache, logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = alterPartitionManager, + isShuttingDown = isShuttingDown, delayedProducePurgatoryParam = Some(mockProducePurgatory), delayedFetchPurgatoryParam = Some(mockFetchPurgatory), delayedDeleteRecordsPurgatoryParam = Some(mockDeleteRecordsPurgatory), @@ -3868,10 +3951,12 @@ class ReplicaManagerTest { val foo2 = new TopicPartition("foo", 2) val mockReplicaFetcherManager = mock(classOf[ReplicaFetcherManager]) + val isShuttingDown = new AtomicBoolean(false) val replicaManager = setupReplicaManagerWithMockedPurgatories( timer = new MockTimer(time), brokerId = localId, - mockReplicaFetcherManager = Some(mockReplicaFetcherManager) + mockReplicaFetcherManager = Some(mockReplicaFetcherManager), + isShuttingDown = isShuttingDown ) try { @@ -3940,6 +4025,10 @@ class ReplicaManagerTest { reset(mockReplicaFetcherManager) + // The broker transitions to SHUTTING_DOWN state. This should not have + // any impact in KRaft mode. + isShuttingDown.set(true) + // The replica begins the controlled shutdown. replicaManager.beginControlledShutdown() @@ -4166,11 +4255,14 @@ class ReplicaManagerTest { class MockReplicaSelector extends ReplicaSelector { private val selectionCount = new AtomicLong() + private var partitionViewArgument: Option[PartitionView] = None def getSelectionCount: Long = selectionCount.get + def getPartitionViewArgument: Option[PartitionView] = partitionViewArgument override def select(topicPartition: TopicPartition, clientMetadata: ClientMetadata, partitionView: PartitionView): Optional[ReplicaView] = { selectionCount.incrementAndGet() + partitionViewArgument = Some(partitionView) Optional.of(partitionView.leader) } } diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 6d17e937824d5..82c19949e3bcf 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -320,7 +320,7 @@ class RequestQuotaTest extends BaseRequestTest { ) ) case ApiKeys.OFFSET_FETCH => - new OffsetFetchRequest.Builder("test-group", false, List(tp).asJava, false) + new OffsetFetchRequest.Builder(Map("test-group"-> List(tp).asJava).asJava, false, false) case ApiKeys.FIND_COORDINATOR => new FindCoordinatorRequest.Builder( diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 96aeac5fa61ec..70554d9427c2f 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -38,7 +38,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.serialization.{IntegerDeserializer, IntegerSerializer, StringDeserializer, StringSerializer} import org.apache.kafka.common.utils.Time import org.apache.kafka.metadata.BrokerState -import org.junit.jupiter.api.{BeforeEach, Disabled, Test, TestInfo, Timeout} +import org.junit.jupiter.api.{BeforeEach, Disabled, TestInfo, Timeout} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.function.Executable import org.junit.jupiter.params.ParameterizedTest @@ -251,9 +251,11 @@ class ServerShutdownTest extends KafkaServerTestHarness { } // Verify that if controller is in the midst of processing a request, shutdown completes - // without waiting for request timeout. - @Test - def testControllerShutdownDuringSend(): Unit = { + // without waiting for request timeout. Since this involves LeaderAndIsr request, it is + // ZK-only for now. + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk")) + def testControllerShutdownDuringSend(quorum: String): Unit = { val securityProtocol = SecurityProtocol.PLAINTEXT val listenerName = ListenerName.forSecurityProtocol(securityProtocol) diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala index f75823a02928b..ec1ee666820ef 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala @@ -20,24 +20,32 @@ package kafka.server.metadata import java.util import java.util.concurrent.atomic.AtomicReference import java.util.{Collections, Optional} - -import org.apache.kafka.common.metadata.{PartitionChangeRecord, PartitionRecord, RegisterBrokerRecord, TopicRecord} +import org.apache.kafka.common.metadata.{FeatureLevelRecord, PartitionChangeRecord, PartitionRecord, RegisterBrokerRecord, TopicRecord} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{Endpoint, Uuid} import org.apache.kafka.image.{MetadataDelta, MetadataImage} import org.apache.kafka.metadata.{BrokerRegistration, RecordTestUtils, VersionRange} -import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} +import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} -import org.junit.jupiter.api.Test +import org.junit.jupiter.api.{AfterEach, Test} import scala.jdk.CollectionConverters._ class BrokerMetadataListenerTest { + private val metadataLoadingFaultHandler = new MockFaultHandler("metadata loading") + + @AfterEach + def verifyNoFaults(): Unit = { + metadataLoadingFaultHandler.maybeRethrowFirstException() + } + private def newBrokerMetadataListener( metrics: BrokerServerMetrics = BrokerServerMetrics(new Metrics()), snapshotter: Option[MetadataSnapshotter] = None, maxBytesBetweenSnapshots: Long = 1000000L, + faultHandler: FaultHandler = metadataLoadingFaultHandler ): BrokerMetadataListener = { new BrokerMetadataListener( brokerId = 0, @@ -45,7 +53,8 @@ class BrokerMetadataListenerTest { threadNamePrefix = None, maxBytesBetweenSnapshots = maxBytesBetweenSnapshots, snapshotter = snapshotter, - brokerMetrics = metrics + brokerMetrics = metrics, + _metadataLoadingFaultHandler = faultHandler ) } @@ -74,10 +83,12 @@ class BrokerMetadataListenerTest { ) ) val imageRecords = listener.getImageRecords().get() - assertEquals(1, imageRecords.size()) + assertEquals(0, imageRecords.size()) assertEquals(100L, listener.highestMetadataOffset) assertEquals(0L, metrics.lastAppliedRecordOffset.get) assertEquals(0L, metrics.lastAppliedRecordTimestamp.get) + assertEquals(0L, metrics.metadataLoadErrorCount.get) + assertEquals(0L, metrics.metadataApplyErrorCount.get) val fencedTimestamp = 500L val fencedLastOffset = 200L @@ -111,6 +122,8 @@ class BrokerMetadataListenerTest { assertEquals(fencedLastOffset, metrics.lastAppliedRecordOffset.get) assertEquals(fencedTimestamp, metrics.lastAppliedRecordTimestamp.get) + assertEquals(0L, metrics.metadataLoadErrorCount.get) + assertEquals(0L, metrics.metadataApplyErrorCount.get) } finally { listener.close() } @@ -156,6 +169,7 @@ class BrokerMetadataListenerTest { } private val FOO_ID = Uuid.fromString("jj1G9utnTuCegi_gpnRgYw") + private val BAR_ID = Uuid.fromString("SzN5j0LvSEaRIJHrxfMAlg") private def generateManyRecords(listener: BrokerMetadataListener, endOffset: Long): Unit = { @@ -180,6 +194,27 @@ class BrokerMetadataListenerTest { listener.getImageRecords().get() } + private def generateBadRecords(listener: BrokerMetadataListener, + endOffset: Long): Unit = { + listener.handleCommit( + RecordTestUtils.mockBatchReader( + endOffset, + 0, + util.Arrays.asList( + new ApiMessageAndVersion(new PartitionChangeRecord(). + setPartitionId(0). + setTopicId(BAR_ID). + setRemovingReplicas(Collections.singletonList(1)), 0.toShort), + new ApiMessageAndVersion(new PartitionChangeRecord(). + setPartitionId(0). + setTopicId(BAR_ID). + setRemovingReplicas(Collections.emptyList()), 0.toShort) + ) + ) + ) + listener.getImageRecords().get() + } + @Test def testHandleCommitsWithNoSnapshotterDefined(): Unit = { val listener = newBrokerMetadataListener(maxBytesBetweenSnapshots = 1000L) @@ -240,6 +275,76 @@ class BrokerMetadataListenerTest { } } + @Test + def testNotSnapshotAfterMetadataVersionChangeBeforePublishing(): Unit = { + val snapshotter = new MockMetadataSnapshotter() + val listener = newBrokerMetadataListener(snapshotter = Some(snapshotter), + maxBytesBetweenSnapshots = 1000L) + + updateFeature(listener, feature = MetadataVersion.FEATURE_NAME, MetadataVersion.latest.featureLevel(), 100L) + listener.getImageRecords().get() + assertEquals(-1L, snapshotter.activeSnapshotOffset, "We won't generate snapshot on metadata version change before starting publishing") + } + + @Test + def testSnapshotAfterMetadataVersionChangeWhenStarting(): Unit = { + val snapshotter = new MockMetadataSnapshotter() + val listener = newBrokerMetadataListener(snapshotter = Some(snapshotter), + maxBytesBetweenSnapshots = 1000L) + + val endOffset = 100L + updateFeature(listener, feature = MetadataVersion.FEATURE_NAME, MetadataVersion.latest.featureLevel(), endOffset) + listener.startPublishing(new MockMetadataPublisher()).get() + assertEquals(endOffset, snapshotter.activeSnapshotOffset, "We should try to generate snapshot when starting publishing") + } + + @Test + def testSnapshotAfterMetadataVersionChange(): Unit = { + val snapshotter = new MockMetadataSnapshotter() + val listener = newBrokerMetadataListener(snapshotter = Some(snapshotter), + maxBytesBetweenSnapshots = 1000L) + listener.startPublishing(new MockMetadataPublisher()).get() + + val endOffset = 100L + updateFeature(listener, feature = MetadataVersion.FEATURE_NAME, (MetadataVersion.latest().featureLevel() - 1).toShort, endOffset) + // Waiting for the metadata version update to get processed + listener.getImageRecords().get() + assertEquals(endOffset, snapshotter.activeSnapshotOffset, "We should generate snapshot on feature update") + } + + @Test + def testNoSnapshotAfterError(): Unit = { + val snapshotter = new MockMetadataSnapshotter() + val faultHandler = new MockFaultHandler("metadata loading") + + val listener = newBrokerMetadataListener( + snapshotter = Some(snapshotter), + maxBytesBetweenSnapshots = 1000L, + faultHandler = faultHandler) + try { + val brokerIds = 0 to 3 + + registerBrokers(listener, brokerIds, endOffset = 100L) + createTopicWithOnePartition(listener, replicas = brokerIds, endOffset = 200L) + listener.getImageRecords().get() + assertEquals(200L, listener.highestMetadataOffset) + assertEquals(-1L, snapshotter.prevCommittedOffset) + assertEquals(-1L, snapshotter.activeSnapshotOffset) + + // Append invalid records that will normally trigger a snapshot + generateBadRecords(listener, 1000L) + assertEquals(-1L, snapshotter.prevCommittedOffset) + assertEquals(-1L, snapshotter.activeSnapshotOffset) + + // Generate some records that will not throw an error, verify still no snapshots + generateManyRecords(listener, 2000L) + assertEquals(-1L, snapshotter.prevCommittedOffset) + assertEquals(-1L, snapshotter.activeSnapshotOffset) + } finally { + listener.close() + } + } + private def registerBrokers( listener: BrokerMetadataListener, brokerIds: Iterable[Int], @@ -285,4 +390,23 @@ class BrokerMetadataListenerTest { ) } + private def updateFeature( + listener: BrokerMetadataListener, + feature: String, + version: Short, + endOffset: Long + ): Unit = { + listener.handleCommit( + RecordTestUtils.mockBatchReader( + endOffset, + 0, + util.Arrays.asList( + new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(feature). + setFeatureLevel(version), 0.toShort) + ) + ) + ) + } + } diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala index 6742530ef5144..472d7ef550b44 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala @@ -15,19 +15,17 @@ * limitations under the License. */ -package unit.kafka.server.metadata +package kafka.server.metadata -import java.util.Collections.{singleton, singletonMap} +import java.util.Collections.{singleton, singletonList, singletonMap} import java.util.Properties import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} - import kafka.log.UnifiedLog -import kafka.server.KafkaConfig -import kafka.server.metadata.BrokerMetadataPublisher +import kafka.server.{BrokerServer, KafkaConfig} import kafka.testkit.{KafkaClusterTestKit, TestKitNodes} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET -import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ConfigEntry} +import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ConfigEntry, NewTopic} import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource.Type.BROKER import org.apache.kafka.common.utils.Exit @@ -35,10 +33,12 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.image.{MetadataImageTest, TopicImage, TopicsImage} import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.PartitionRegistration -import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} +import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler} +import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertTrue} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Test} import org.mockito.ArgumentMatchers.any import org.mockito.Mockito +import org.mockito.Mockito.doThrow import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -176,6 +176,26 @@ class BrokerMetadataPublisherTest { new TopicsImage(idsMap.asJava, namesMap.asJava) } + private def newMockPublisher( + broker: BrokerServer, + errorHandler: FaultHandler = new MockFaultHandler("publisher") + ): BrokerMetadataPublisher = { + Mockito.spy(new BrokerMetadataPublisher( + conf = broker.config, + metadataCache = broker.metadataCache, + logManager = broker.logManager, + replicaManager = broker.replicaManager, + groupCoordinator = broker.groupCoordinator, + txnCoordinator = broker.transactionCoordinator, + clientQuotaMetadataManager = broker.clientQuotaMetadataManager, + dynamicConfigHandlers = broker.dynamicConfigHandlers.toMap, + _authorizer = Option.empty, + errorHandler, + errorHandler + )) + } + + @Disabled @Test def testReloadUpdatedFilesWithoutConfigChange(): Unit = { val cluster = new KafkaClusterTestKit.Builder( @@ -187,17 +207,7 @@ class BrokerMetadataPublisherTest { cluster.startup() cluster.waitForReadyBrokers() val broker = cluster.brokers().values().iterator().next() - val publisher = Mockito.spy(new BrokerMetadataPublisher( - conf = broker.config, - metadataCache = broker.metadataCache, - logManager = broker.logManager, - replicaManager = broker.replicaManager, - groupCoordinator = broker.groupCoordinator, - txnCoordinator = broker.transactionCoordinator, - clientQuotaMetadataManager = broker.clientQuotaMetadataManager, - dynamicConfigHandlers = broker.dynamicConfigHandlers.toMap, - _authorizer = Option.empty - )) + val publisher = newMockPublisher(broker) val numTimesReloadCalled = new AtomicInteger(0) Mockito.when(publisher.reloadUpdatedFilesWithoutConfigChange(any[Properties]())). thenAnswer(new Answer[Unit]() { @@ -227,4 +237,39 @@ class BrokerMetadataPublisherTest { cluster.close() } } + + @Test + def testExceptionInUpdateCoordinator(): Unit = { + val errorHandler = new MockFaultHandler("publisher") + val cluster = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(1).build()). + setMetadataFaultHandler(errorHandler).build() + try { + cluster.format() + cluster.startup() + cluster.waitForReadyBrokers() + val broker = cluster.brokers().values().iterator().next() + TestUtils.retry(60000) { + assertNotNull(broker.metadataPublisher) + } + val publisher = Mockito.spy(broker.metadataPublisher) + doThrow(new RuntimeException("injected failure")).when(publisher).updateCoordinator(any(), any(), any(), any(), any()) + broker.metadataListener.alterPublisher(publisher).get() + val admin = Admin.create(cluster.clientProperties()) + try { + admin.createTopics(singletonList(new NewTopic("foo", 1, 1.toShort))).all().get() + } finally { + admin.close() + } + TestUtils.retry(60000) { + assertTrue(Option(errorHandler.firstException()). + flatMap(e => Option(e.getMessage())).getOrElse("(none)").contains("injected failure")) + } + } finally { + errorHandler.setIgnore(true) + cluster.close() + } + } } diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala index 82426611425a1..ff2326c92fa37 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala @@ -20,8 +20,8 @@ package kafka.server.metadata import java.nio.ByteBuffer import java.util.Optional import java.util.concurrent.{CompletableFuture, CountDownLatch} - import org.apache.kafka.common.memory.MemoryPool +import org.apache.kafka.common.metadata.FenceBrokerRecord import org.apache.kafka.common.protocol.ByteBufferAccessor import org.apache.kafka.common.record.{CompressionType, MemoryRecords} import org.apache.kafka.common.utils.Time @@ -34,6 +34,9 @@ import org.apache.kafka.snapshot.{MockRawSnapshotWriter, RecordsSnapshotWriter, import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.Test +import java.util +import java.util.Arrays.asList +import scala.compat.java8.OptionConverters._ class BrokerMetadataSnapshotterTest { @Test @@ -48,7 +51,7 @@ class BrokerMetadataSnapshotterTest { override def build(committedOffset: Long, committedEpoch: Int, - lastContainedLogTime: Long): SnapshotWriter[ApiMessageAndVersion] = { + lastContainedLogTime: Long): Option[SnapshotWriter[ApiMessageAndVersion]] = { val offsetAndEpoch = new OffsetAndEpoch(committedOffset, committedEpoch) RecordsSnapshotWriter.createWithHeader( () => { @@ -62,7 +65,7 @@ class BrokerMetadataSnapshotterTest { lastContainedLogTime, CompressionType.NONE, MetadataRecordSerde.INSTANCE - ).get(); + ).asScala } def consumeSnapshotBuffer(committedOffset: Long, committedEpoch: Int)(buffer: ByteBuffer): Unit = { @@ -104,4 +107,31 @@ class BrokerMetadataSnapshotterTest { snapshotter.close() } } + + class MockSnapshotWriter extends SnapshotWriter[ApiMessageAndVersion] { + val batches = new util.ArrayList[util.List[ApiMessageAndVersion]] + override def snapshotId(): OffsetAndEpoch = new OffsetAndEpoch(0, 0) + override def lastContainedLogOffset(): Long = 0 + override def lastContainedLogEpoch(): Int = 0 + override def isFrozen: Boolean = false + override def append(batch: util.List[ApiMessageAndVersion]): Unit = batches.add(batch) + override def freeze(): Unit = {} + override def close(): Unit = {} + } + + @Test + def testRecordListConsumer(): Unit = { + val writer = new MockSnapshotWriter() + val consumer = new RecordListConsumer(3, writer) + val m = new ApiMessageAndVersion(new FenceBrokerRecord().setId(1).setEpoch(1), 0.toShort) + consumer.accept(asList(m, m)) + assertEquals(asList(asList(m, m)), writer.batches) + consumer.accept(asList(m)) + assertEquals(asList(asList(m, m), asList(m)), writer.batches) + consumer.accept(asList(m, m, m, m)) + assertEquals(asList(asList(m, m), asList(m), asList(m, m, m), asList(m)), writer.batches) + consumer.accept(asList(m, m, m, m, m, m, m, m)) + assertEquals(asList(asList(m, m), asList(m), asList(m, m, m), asList(m), asList(m, m, m), asList(m, m, m), asList(m, m)), + writer.batches) + } } diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index 0e11471527ae3..cbffbf0292e16 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -191,18 +191,26 @@ Found problem: @Test def testDefaultMetadataVersion(): Unit = { val namespace = StorageTool.parseArguments(Array("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ")) - val mv = StorageTool.getMetadataVersion(namespace) + val mv = StorageTool.getMetadataVersion(namespace, defaultVersionString = None) assertEquals(MetadataVersion.latest().featureLevel(), mv.featureLevel(), "Expected the default metadata.version to be the latest version") } + @Test + def testConfiguredMetadataVersion(): Unit = { + val namespace = StorageTool.parseArguments(Array("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ")) + val mv = StorageTool.getMetadataVersion(namespace, defaultVersionString = Some(MetadataVersion.IBP_3_3_IV2.toString)) + assertEquals(MetadataVersion.IBP_3_3_IV2.featureLevel(), mv.featureLevel(), + "Expected the default metadata.version to be 3.3-IV2") + } + @Test def testMetadataVersionFlags(): Unit = { def parseMetadataVersion(strings: String*): MetadataVersion = { var args = mutable.Seq("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ") args ++= strings val namespace = StorageTool.parseArguments(args.toArray) - StorageTool.getMetadataVersion(namespace) + StorageTool.getMetadataVersion(namespace, defaultVersionString = None) } var mv = parseMetadataVersion("--release-version", "3.0") diff --git a/core/src/test/scala/unit/kafka/utils/PasswordEncoderTest.scala b/core/src/test/scala/unit/kafka/utils/PasswordEncoderTest.scala index 0a5d5ac029814..50cdceabbca66 100755 --- a/core/src/test/scala/unit/kafka/utils/PasswordEncoderTest.scala +++ b/core/src/test/scala/unit/kafka/utils/PasswordEncoderTest.scala @@ -30,7 +30,7 @@ class PasswordEncoderTest { @Test def testEncodeDecode(): Unit = { - val encoder = new PasswordEncoder(new Password("password-encoder-secret"), + val encoder = PasswordEncoder.encrypting(new Password("password-encoder-secret"), None, Defaults.PasswordEncoderCipherAlgorithm, Defaults.PasswordEncoderKeyLength, @@ -54,7 +54,7 @@ class PasswordEncoderTest { @Test def testEncoderConfigChange(): Unit = { - val encoder = new PasswordEncoder(new Password("password-encoder-secret"), + val encoder = PasswordEncoder.encrypting(new Password("password-encoder-secret"), Some("PBKDF2WithHmacSHA1"), "DES/CBC/PKCS5Padding", 64, @@ -68,7 +68,7 @@ class PasswordEncoderTest { assertEquals("DES/CBC/PKCS5Padding", encodedMap(PasswordEncoder.CipherAlgorithmProp)) // Test that decoding works even if PasswordEncoder algorithm, iterations etc. are altered - val decoder = new PasswordEncoder(new Password("password-encoder-secret"), + val decoder = PasswordEncoder.encrypting(new Password("password-encoder-secret"), Some("PBKDF2WithHmacSHA1"), "AES/CBC/PKCS5Padding", 128, @@ -76,7 +76,7 @@ class PasswordEncoderTest { assertEquals(password, decoder.decode(encoded).value) // Test that decoding fails if secret is altered - val decoder2 = new PasswordEncoder(new Password("secret-2"), + val decoder2 = PasswordEncoder.encrypting(new Password("secret-2"), Some("PBKDF2WithHmacSHA1"), "AES/CBC/PKCS5Padding", 128, @@ -92,7 +92,7 @@ class PasswordEncoderTest { def testEncodeDecodeAlgorithms(): Unit = { def verifyEncodeDecode(keyFactoryAlg: Option[String], cipherAlg: String, keyLength: Int): Unit = { - val encoder = new PasswordEncoder(new Password("password-encoder-secret"), + val encoder = PasswordEncoder.encrypting(new Password("password-encoder-secret"), keyFactoryAlg, cipherAlg, keyLength, diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 5a8d43795ae8a..833530e9cbf80 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -47,7 +47,7 @@ import org.apache.kafka.clients.admin._ import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.consumer.internals.AbstractCoordinator import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} -import org.apache.kafka.common.TopicIdPartition +import org.apache.kafka.common.{KafkaFuture, Node, TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBinding, AclBindingFilter} import org.apache.kafka.common.config.{ConfigException, ConfigResource} import org.apache.kafka.common.config.ConfigResource.Type.TOPIC @@ -67,7 +67,6 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, KafkaPrincipalSerd import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, Deserializer, IntegerSerializer, Serializer} import org.apache.kafka.common.utils.Utils._ import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.common.{KafkaFuture, TopicPartition, Uuid} import org.apache.kafka.controller.QuorumController import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer} import org.apache.kafka.server.common.MetadataVersion @@ -210,14 +209,14 @@ object TestUtils extends Logging { enableToken: Boolean = false, numPartitions: Int = 1, defaultReplicationFactor: Short = 1, - startingIdNumber: Int = 0 - ): Seq[Properties] = { + startingIdNumber: Int = 0, + enableFetchFromFollower: Boolean = false): Seq[Properties] = { val endingIdNumber = startingIdNumber + numConfigs - 1 (startingIdNumber to endingIdNumber).map { node => createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, RandomPort, interBrokerSecurityProtocol, trustStoreFile, saslProperties, enablePlaintext = enablePlaintext, enableSsl = enableSsl, enableSaslPlaintext = enableSaslPlaintext, enableSaslSsl = enableSaslSsl, rack = rackInfo.get(node), logDirCount = logDirCount, enableToken = enableToken, - numPartitions = numPartitions, defaultReplicationFactor = defaultReplicationFactor) + numPartitions = numPartitions, defaultReplicationFactor = defaultReplicationFactor, enableFetchFromFollower = enableFetchFromFollower) } } @@ -276,7 +275,8 @@ object TestUtils extends Logging { logDirCount: Int = 1, enableToken: Boolean = false, numPartitions: Int = 1, - defaultReplicationFactor: Short = 1): Properties = { + defaultReplicationFactor: Short = 1, + enableFetchFromFollower: Boolean = false): Properties = { def shouldEnable(protocol: SecurityProtocol) = interBrokerSecurityProtocol.fold(false)(_ == protocol) val protocolAndPorts = ArrayBuffer[(SecurityProtocol, Int)]() @@ -358,6 +358,11 @@ object TestUtils extends Logging { props.put(KafkaConfig.NumPartitionsProp, numPartitions.toString) props.put(KafkaConfig.DefaultReplicationFactorProp, defaultReplicationFactor.toString) + if (enableFetchFromFollower) { + props.put(KafkaConfig.RackProp, nodeId.toString) + props.put(KafkaConfig.ReplicaSelectorClassProp, "org.apache.kafka.common.replica.RackAwareReplicaSelector") + } + props } @@ -511,7 +516,7 @@ object TestUtils extends Logging { topic: String, numPartitions: Int = 1, replicationFactor: Int = 1, - servers: Seq[KafkaServer], + servers: Seq[KafkaBroker], topicConfig: Properties = new Properties): scala.collection.immutable.Map[Int, Int] = { val adminZkClient = new AdminZkClient(zkClient) // create topic @@ -543,7 +548,7 @@ object TestUtils extends Logging { def createTopic(zkClient: KafkaZkClient, topic: String, partitionReplicaAssignment: collection.Map[Int, Seq[Int]], - servers: Seq[KafkaServer]): scala.collection.immutable.Map[Int, Int] = { + servers: Seq[KafkaBroker]): scala.collection.immutable.Map[Int, Int] = { createTopic(zkClient, topic, partitionReplicaAssignment, servers, new Properties()) } @@ -555,7 +560,7 @@ object TestUtils extends Logging { def createTopic(zkClient: KafkaZkClient, topic: String, partitionReplicaAssignment: collection.Map[Int, Seq[Int]], - servers: Seq[KafkaServer], + servers: Seq[KafkaBroker], topicConfig: Properties): scala.collection.immutable.Map[Int, Int] = { val adminZkClient = new AdminZkClient(zkClient) // create topic @@ -583,7 +588,7 @@ object TestUtils extends Logging { * Create the consumer offsets/group metadata topic and wait until the leader is elected and metadata is propagated * to all brokers. */ - def createOffsetsTopic(zkClient: KafkaZkClient, servers: Seq[KafkaServer]): Unit = { + def createOffsetsTopic(zkClient: KafkaZkClient, servers: Seq[KafkaBroker]): Unit = { val server = servers.head createTopic(zkClient, Topic.GROUP_METADATA_TOPIC_NAME, server.config.getInt(KafkaConfig.OffsetsTopicPartitionsProp), @@ -871,8 +876,59 @@ object TestUtils extends Logging { * LeaderDuringDelete). * @throws AssertionError if the expected condition is not true within the timeout. */ - def waitUntilLeaderIsElectedOrChanged(zkClient: KafkaZkClient, topic: String, partition: Int, timeoutMs: Long = 30000L, - oldLeaderOpt: Option[Int] = None, newLeaderOpt: Option[Int] = None): Int = { + def waitUntilLeaderIsElectedOrChanged( + zkClient: KafkaZkClient, + topic: String, + partition: Int, + timeoutMs: Long = 30000L, + oldLeaderOpt: Option[Int] = None, + newLeaderOpt: Option[Int] = None + ): Int = { + def getPartitionLeader(topic: String, partition: Int): Option[Int] = { + zkClient.getLeaderForPartition(new TopicPartition(topic, partition)) + } + doWaitUntilLeaderIsElectedOrChanged(getPartitionLeader, topic, partition, timeoutMs, oldLeaderOpt, newLeaderOpt) + } + + /** + * If neither oldLeaderOpt nor newLeaderOpt is defined, wait until the leader of a partition is elected. + * If oldLeaderOpt is defined, it waits until the new leader is different from the old leader. + * If newLeaderOpt is defined, it waits until the new leader becomes the expected new leader. + * + * @return The new leader (note that negative values are used to indicate conditions like NoLeader and + * LeaderDuringDelete). + * @throws AssertionError if the expected condition is not true within the timeout. + */ + def waitUntilLeaderIsElectedOrChangedWithAdmin( + admin: Admin, + topic: String, + partition: Int, + timeoutMs: Long = 30000L, + oldLeaderOpt: Option[Int] = None, + newLeaderOpt: Option[Int] = None + ): Int = { + def getPartitionLeader(topic: String, partition: Int): Option[Int] = { + admin.describeTopics(Collections.singletonList(topic)).allTopicNames().get().get(topic).partitions().asScala. + find(_.partition() == partition). + flatMap { p => + if (p.leader().id() == Node.noNode().id()) { + None + } else { + Some(p.leader().id()) + } + } + } + doWaitUntilLeaderIsElectedOrChanged(getPartitionLeader, topic, partition, timeoutMs, oldLeaderOpt, newLeaderOpt) + } + + private def doWaitUntilLeaderIsElectedOrChanged( + getPartitionLeader: (String, Int) => Option[Int], + topic: String, + partition: Int, + timeoutMs: Long, + oldLeaderOpt: Option[Int], + newLeaderOpt: Option[Int] + ): Int = { require(!(oldLeaderOpt.isDefined && newLeaderOpt.isDefined), "Can't define both the old and the new leader") val startTime = System.currentTimeMillis() val topicPartition = new TopicPartition(topic, partition) @@ -884,7 +940,7 @@ object TestUtils extends Logging { var electedOrChangedLeader: Option[Int] = None while (electedOrChangedLeader.isEmpty && System.currentTimeMillis() < startTime + timeoutMs) { // check if leader is elected - leader = zkClient.getLeaderForPartition(topicPartition) + leader = getPartitionLeader(topic, partition) leader match { case Some(l) => (newLeaderOpt, oldLeaderOpt) match { case (Some(newLeader), _) if newLeader == l => @@ -1043,18 +1099,19 @@ object TestUtils extends Logging { * otherwise difficult to poll for. `computeUntilTrue` and `waitUntilTrue` should be preferred in cases where we can * easily wait on a condition before evaluating the assertions. */ - def tryUntilNoAssertionError(waitTime: Long = JTestUtils.DEFAULT_MAX_WAIT_MS, pause: Long = 100L)(assertions: => Unit) = { - val (error, success) = TestUtils.computeUntilTrue({ + def tryUntilNoAssertionError[T](waitTime: Long = JTestUtils.DEFAULT_MAX_WAIT_MS, pause: Long = 100L)(assertions: => T): T = { + val (either, success) = TestUtils.computeUntilTrue({ try { - assertions - None + val res = assertions + Left(res) } catch { - case ae: AssertionError => Some(ae) + case ae: AssertionError => Right(ae) } - }, waitTime = waitTime, pause = pause)(_.isEmpty) + }, waitTime = waitTime, pause = pause)(_.isLeft) - if (!success) { - throw error.get + either match { + case Left(res) => res + case Right(err) => throw err } } @@ -1281,13 +1338,14 @@ object TestUtils extends Logging { configRepository: ConfigRepository = new MockConfigRepository, cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false), time: MockTime = new MockTime(), - interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest): LogManager = { + interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest, + recoveryThreadsPerDataDir: Int = 4): LogManager = { new LogManager(logDirs = logDirs.map(_.getAbsoluteFile), initialOfflineDirs = Array.empty[File], configRepository = configRepository, initialDefaultConfig = defaultConfig, cleanerConfig = cleanerConfig, - recoveryThreadsPerDataDir = 4, + recoveryThreadsPerDataDir = recoveryThreadsPerDataDir, flushCheckMs = 1000L, flushRecoveryOffsetCheckpointMs = 10000L, flushStartOffsetCheckpointMs = 10000L, @@ -1976,7 +2034,10 @@ object TestUtils extends Logging { fail("Expected illegal configuration but instead it was legal") } catch { case caught @ (_: ConfigException | _: IllegalArgumentException) => - assertTrue(caught.getMessage.contains(expectedExceptionContainsText)) + assertTrue( + caught.getMessage.contains(expectedExceptionContainsText), + s""""${caught.getMessage}" doesn't contain "$expectedExceptionContainsText"""" + ) } } @@ -2260,4 +2321,4 @@ object TestUtils extends Logging { timedOut.set(true) } } -} \ No newline at end of file +} diff --git a/docs/configuration.html b/docs/configuration.html index ceb671ca7500d..c2f342f2ee1ab 100644 --- a/docs/configuration.html +++ b/docs/configuration.html @@ -20,13 +20,22 @@

3.1 Broker Configs

- The essential configurations are the following: + For ZooKeeper clusters, brokers must have the following configuration:
    -
  • broker.id -
  • log.dirs -
  • zookeeper.connect +
  • broker.id
  • +
  • log.dirs
  • +
  • zookeeper.connect
+ For KRaft clusters, brokers and controllers must have the following configurations: +
    +
  • node.id
  • +
  • log.dirs
  • +
  • process.roles
  • +
+ + On KRaft brokers, if broker.id is set, it must be equal to node.id. + Topic-level configurations and defaults are discussed in more detail below. @@ -62,13 +71,16 @@

All configs that are configurable at cluster level may also be configured at per-broker level (e.g. for testing). If a config value is defined at different levels, the following order of precedence is used:
    -
  • Dynamic per-broker config stored in ZooKeeper
  • -
  • Dynamic cluster-wide default config stored in ZooKeeper
  • -
  • Static broker config from server.properties
  • +
  • Dynamic per-broker configs
  • +
  • Dynamic cluster-wide default configs
  • +
  • Static broker configs from server.properties
  • Kafka default, see broker configs
-

Updating Password Configs Dynamically
+ Dynamic configs are stored in Kafka as cluster metadata. In ZooKeeper mode, dynamic configs are stored in ZooKeeper. + In KRaft mode, dynamic configs are stored as records in the metadata log. + +
Updating Password Configs Dynamically (ZooKeeper-only)

Password config values that are dynamically updated are encrypted before storing in ZooKeeper. The broker config password.encoder.secret must be configured in server.properties to enable dynamic update of password configs. The secret may be different on different brokers.

@@ -159,12 +171,18 @@
Updating Default Topic Configuration
From Kafka version 2.0.0 onwards, unclean leader election is automatically enabled by the controller when the config unclean.leader.election.enable is dynamically updated. - In Kafka version 1.1.x, changes to unclean.leader.election.enable take effect only when a new controller is elected. - Controller re-election may be forced by running: + In Kafka version 1.1.x, changes to unclean.leader.election.enable take effect only when a new controller + is elected. + + In ZooKeeper mode, a Controller re-election may be forced by removing the Controller's ZNode. This is + done using the zookeeper-shell.sh utility included in the "bin" directory.
> bin/zookeeper-shell.sh localhost
   rmr /controller
+ In KRaft mode, the way to force a Controller re-election is to terminate the active controller node. Since KRaft + controllers do not host partitions, they are generally very quick to restart. +
Updating Log Cleaner Configs
Log cleaner configs may be updated dynamically at cluster-default level used by all brokers. The changes take effect on the next iteration of log cleaning. One or more of these configs may be updated: diff --git a/docs/connect.html b/docs/connect.html index d13d25d31393c..5c3a21e99a5d1 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -327,7 +327,7 @@

REST API

  • GET /- return basic information about the Kafka Connect cluster such as the version of the Connect worker that serves the REST request (including git commit ID of the source code) and the Kafka cluster ID that is connected to. -

    For the complete specification of the REST API, see the OpenAPI documentation

    +

    For the complete specification of the REST API, see the OpenAPI documentation

    Error Reporting in Connect

    diff --git a/docs/design.html b/docs/design.html index 6e32b2d7f6fdf..9485ab9c5005a 100644 --- a/docs/design.html +++ b/docs/design.html @@ -322,18 +322,33 @@

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

    + What is meant by an "active session" depends on the cluster configuration. For KRaft clusters, an active session is maintained by + sending periodic heartbeats to the controller. If the controller fails to receive a heartbeat before the timeout configured by + broker.session.timeout.ms expires, then the node is considered offline. +

    + For clusters using Zookeeper, liveness is determined indirectly through the existence of an ephemeral node which is created by the broker on + initialization of its Zookeeper session. If the broker loses its session after failing to send heartbeats to Zookeeper before expiration of + zookeeper.session.timeout.ms, then the node gets deleted. The controller would then notice the node deletion through a Zookeeper watch + and mark the broker offline. +

    + We refer to nodes satisfying these two conditions as being "in sync" to avoid the vagueness of "alive" or "failed". The leader keeps track of the set of "in sync" replicas, + which is known as the ISR. If either of these conditions fail to be satisified, then the broker will be removed from the ISR. For example, + if a follower dies, then the controller will notice the failure through the loss of its session, and will remove the broker from the ISR. + On the other hand, if the follower lags too far behind the leader but still has an active session, then the leader can also remove it from the ISR. + The determination of lagging replicas is controlled through the replica.lag.time.max.ms configuration. + Replicas that cannot catch up to the end of the log on the leader within the max time set by this configuration are removed from the ISR.

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

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

    Replicated Logs: Q expensive approach is not used for the data itself.

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

    @@ -442,9 +457,10 @@

    < share of its partitions.

    It is also important to optimize the leadership election process as that is the critical window of unavailability. A naive implementation of leader election would end up running an election per partition for all - partitions a node hosted when that node failed. Instead, we elect one of the brokers as the "controller". This controller detects failures at the broker level and is responsible for changing the leader of all - affected partitions in a failed broker. The result is that we are able to batch together many of the required leadership change notifications which makes the election process far cheaper and faster for a large number - of partitions. If the controller fails, one of the surviving brokers will become the new controller. + partitions a node hosted when that node failed. As discussed above in the section on replication, Kafka clusters have a special role known as the "controller" which is + responsible for managing the registration of brokers. If the controller detects the failure of a broker, it is responsible for electing one of the remaining members of the ISR to serve as the new leader. + The result is that we are able to batch together many of the required leadership change notifications which makes the election process far cheaper and faster for a large number + of partitions. If the controller itself fails, then another controller will be elected.

    4.8 Log Compaction

    diff --git a/docs/documentation.html b/docs/documentation.html index 07014db7af4b9..0abbae8718168 100644 --- a/docs/documentation.html +++ b/docs/documentation.html @@ -33,7 +33,7 @@

    Documentation

    -

    Kafka 3.2 Documentation

    +

    Kafka 3.3 Documentation

    Prior releases: 0.7.x, 0.8.0, 0.8.1.X, @@ -56,6 +56,7 @@

    Kafka 3.2 Documentation

    2.8.X, 3.0.X. 3.1.X. + 3.2.X.

    1. Getting Started

    1.1 Introduction

    diff --git a/docs/images/creating-streams-iframe-placeholder.png b/docs/images/creating-streams-iframe-placeholder.png new file mode 100644 index 0000000000000..479a830760dd2 Binary files /dev/null and b/docs/images/creating-streams-iframe-placeholder.png differ diff --git a/docs/images/intro_to_streams-iframe-placeholder.png b/docs/images/intro_to_streams-iframe-placeholder.png new file mode 100644 index 0000000000000..462ec036e2da4 Binary files /dev/null and b/docs/images/intro_to_streams-iframe-placeholder.png differ diff --git a/docs/images/transforming_part_1-iframe-placeholder.png b/docs/images/transforming_part_1-iframe-placeholder.png new file mode 100644 index 0000000000000..e959f0e972faf Binary files /dev/null and b/docs/images/transforming_part_1-iframe-placeholder.png differ diff --git a/docs/images/transforming_part_2-iframe-placeholder.png b/docs/images/transforming_part_2-iframe-placeholder.png new file mode 100644 index 0000000000000..008ec16bb4637 Binary files /dev/null and b/docs/images/transforming_part_2-iframe-placeholder.png differ diff --git a/docs/js/templateData.js b/docs/js/templateData.js index 6b773468f962b..ae682150ece83 100644 --- a/docs/js/templateData.js +++ b/docs/js/templateData.js @@ -19,6 +19,6 @@ limitations under the License. var context={ "version": "33", "dotVersion": "3.3", - "fullDotVersion": "3.3.0", + "fullDotVersion": "3.3.2", "scalaVersion": "2.13" }; diff --git a/docs/ops.html b/docs/ops.html index 76fee9cc868f6..c3ac673fdd150 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -1269,11 +1269,11 @@

    Java 8, Java 11, and Java 17 are supported. Note that Java 8 support has been deprecated since Apache Kafka 3.0 and will be removed in Apache Kafka 4.0. Java 11 and later versions perform significantly better if TLS is enabled, so they are highly recommended (they also include a number of other performance improvements: G1GC, CRC32C, Compact Strings, Thread-Local Handshakes and more). - + From a security perspective, we recommend the latest released patch version as older freely available versions have disclosed security vulnerabilities. Typical arguments for running Kafka with OpenJDK-based Java implementations (including Oracle JDK) are: - +
      -Xmx6g -Xms6g -XX:MetaspaceSize=96m -XX:+UseG1GC
       -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:G1HeapRegionSize=16M
       -XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80 -XX:+ExplicitGCInvokesConcurrent
    @@ -1373,6 +1373,27 @@
  • delalloc: Delayed allocation means that the filesystem avoid allocating any blocks until the physical write occurs. This allows ext4 to allocate a large extent instead of smaller pages and helps ensure the data is written sequentially. This feature is great for throughput. It does seem to involve some locking in the filesystem which adds a bit of latency variance. +

    Replace KRaft Controller Disk

    +

    When Kafka is configured to use KRaft, the controllers store the cluster metadata in the directory specified in metadata.log.dir -- or the first log directory, if metadata.log.dir is not configured. See the documentation for metadata.log.dir for details.

    + +

    If the data in the cluster metdata directory is lost either because of hardware failure or the hardware needs to be replaced, care should be taken when provisioning the new controller node. The new controller node should not be formatted and started until the majority of the controllers have all of the committed data. To determine if the majority of the controllers have the committed data, run the kafka-metadata-quorum.sh tool to describe the replication status:

    + +
     > bin/kafka-metadata-quorum.sh --bootstrap-server broker_host:port describe --replication
    + NodeId  LogEndOffset    Lag     LastFetchTimestamp      LastCaughtUpTimestamp   Status
    + 1       25806           0       1662500992757           1662500992757           Leader
    + ...     ...             ...     ...                     ...                     ...
    +  
    + +

    Check and wait until the Lag is small for a majority of the controllers. If the leader's end offset is not increasing, you can wait until the lag is 0 for a majority; otherwise, you can pick the latest leader end offset and wait until all replicas have reached it. Check and wait until the LastFetchTimestamp and LastCaughtUpTimestamp are close to each other for the majority of the controllers. At this point it is safer to format the controller's metadata log directory. This can be done by running the kafka-storage.sh command.

    + +
     > bin/kafka-storage.sh format --cluster-id uuid --config server_properties
    + +

    It is possible for the bin/kafka-storage.sh format command above to fail with a message like Log directory ... is already formatted. This can happend when combined mode is used and only the metadata log directory was lost but not the others. In that case and only in that case, can you run the kafka-storage.sh format command with the --ignore-formatted option.

    + +

    Start the KRaft controller after formatting the log directories.

    + +
     > /bin/kafka-server-start.sh server_properties
    +

    6.8 Monitoring

    Kafka uses Yammer Metrics for metrics reporting in the server. The Java clients use Kafka Metrics, a built-in metrics registry that minimizes transitive dependencies pulled into client applications. Both expose metrics via JMX and can be configured to report stats using pluggable stats reporters to hook up to your monitoring system. @@ -1794,6 +1815,206 @@

    KRaft Monitoring Metrics

    +The set of metrics that allow monitoring of the KRaft quorum and the metadata log.
    +Note that some exposed metrics depend on the role of the node as defined by process.roles +
    KRaft Quorum Monitoring Metrics
    +These metrics are reported on both Controllers and Brokers in a KRaft Cluster +
  • + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Metric/Attribute nameDescriptionMbean name
    Current StateThe current state of this member; possible values are leader, candidate, voted, follower, unattached.kafka.server:type=raft-metrics,name=current-state
    Current LeaderThe current quorum leader's id; -1 indicates unknown.kafka.server:type=raft-metrics,name=current-leader
    Current VotedThe current voted leader's id; -1 indicates not voted for anyone.kafka.server:type=raft-metrics,name=current-vote
    Current EpochThe current quorum epoch.kafka.server:type=raft-metrics,name=current-epoch
    High WatermarkThe high watermark maintained on this member; -1 if it is unknown.kafka.server:type=raft-metrics,name=high-watermark
    Log End OffsetThe current raft log end offset.kafka.server:type=raft-metrics,name=log-end-offset
    Number of Unknown Voter ConnectionsNumber of unknown voters whose connection information is not cached. This value of this metric is always 0.kafka.server:type=raft-metrics,name=number-unknown-voter-connections
    Average Commit LatencyThe average time in milliseconds to commit an entry in the raft log.kafka.server:type=raft-metrics,name=commit-latency-avg
    Maximum Commit LatencyThe maximum time in milliseconds to commit an entry in the raft log.kafka.server:type=raft-metrics,name=commit-latency-max
    Average Election LatencyThe average time in milliseconds spent on electing a new leader.kafka.server:type=raft-metrics,name=election-latency-avg
    Maximum Election LatencyThe maximum time in milliseconds spent on electing a new leader.kafka.server:type=raft-metrics,name=election-latency-max
    Fetch Records RateThe average number of records fetched from the leader of the raft quorum.kafka.server:type=raft-metrics,name=fetch-records-rate
    Append Records RateThe average number of records appended per sec by the leader of the raft quorum.kafka.server:type=raft-metrics,name=append-records-raft
    Average Poll Idle RatioThe average fraction of time the client's poll() is idle as opposed to waiting for the user code to process records.kafka.server:type=raft-metrics,name=poll-idle-ratio-avg
    +

    KRaft Controller Monitoring Metrics
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Metric/Attribute nameDescriptionMbean name
    Active Controller CountThe number of Active Controllers on this node. Valid values are '0' or '1'.kafka.controller:type=KafkaController,name=ActiveControllerCount
    Event Queue Time MsA Histogram of the time in milliseconds that requests spent waiting in the Controller Event Queue.kafka.controller:type=ControllerEventManager,name=EventQueueTimeMs
    Event Queue Processing Time MsA Histogram of the time in milliseconds that requests spent being processed in the Controller Event Queue.kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs
    Fenced Broker CountThe number of fenced brokers as observed by this Controller.kafka.controller:type=KafkaController,name=FencedBrokerCount
    Active Broker CountThe number of fenced brokers as observed by this Controller.kafka.controller:type=KafkaController,name=ActiveBrokerCount
    Global Topic CountThe number of global topics as observed by this Controller.kafka.controller:type=KafkaController,name=GlobalTopicCount
    Global Partition CountThe number of global partitions as observed by this Controller.kafka.controller:type=KafkaController,name=GlobalPartitionCount
    Offline Partition CountThe number of offline topic partitions (non-internal) as observed by this Controller.kafka.controller:type=KafkaController,name=OfflinePartitionCount
    Preferred Replica Imbalance CountThe count of topic partitions for which the leader is not the preferred leader.kafka.controller:type=KafkaController,name=PreferredReplicaImbalanceCount
    Metadata Error CountThe number of times this controller node has encountered an error during metadata log processing.kafka.controller:type=KafkaController,name=MetadataErrorCount
    Last Applied Record OffsetThe offset of the last record from the cluster metadata partition that was applied by the Controller.kafka.controller:type=KafkaController,name=LastAppliedRecordOffset
    Last Committed Record OffsetThe offset of the last record committed to this Controller.kafka.controller:type=KafkaController,name=LastCommittedRecordOffset
    Last Applied Record TimestampThe timestamp of the last record from the cluster metadata partition that was applied by the Controller.kafka.controller:type=KafkaController,name=LastAppliedRecordTimestamp
    Last Applied Record Lag MsThe difference between now and the timestamp of the last record from the cluster metadata partition that was applied by the controller. + For active Controllers the value of this lag is always zero.kafka.controller:type=KafkaController,name=LastAppliedRecordLagMs
    +
    KRaft Broker Monitoring Metrics
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Metric/Attribute nameDescriptionMbean name
    Last Applied Record OffsetThe offset of the last record from the cluster metadata partition that was applied by the brokerkafka.server:type=broker-metadata-metrics,name=last-applied-record-offset
    Last Applied Record TimestampThe timestamp of the last record from the cluster metadata partition that was applied by the broker.kafka.server:type=broker-metadata-metrics,name=last-applied-record-timestamp
    Last Applied Record Lag MsThe difference between now and the timestamp of the last record from the cluster metadata partition that was applied by the brokerkafka.server:type=broker-metadata-metrics,name=last-applied-record-lag-ms
    Metadata Load Error CountThe number of errors encountered by the BrokerMetadataListener while loading the metadata log and generating a new MetadataDelta based on it.kafka.server:type=broker-metadata-metrics,name=metadata-load-error-count
    Metadata Apply Error CountThe number of errors encountered by the BrokerMetadataPublisher while applying a new MetadataImage based on the latest MetadataDelta.kafka.server:type=broker-metadata-metrics,name=metadata-apply-error-count

    Common monitoring metrics for producer/consumer/connect/streams

    The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections. @@ -2112,6 +2333,36 @@

    < The total time an appender waits for space allocation in nanoseconds. kafka.producer:type=producer-metrics,client-id=([-.\w]+) + + flush-time-ns-total + The total time the Producer spent in Producer.flush in nanoseconds. + kafka.producer:type=producer-metrics,client-id=([-.\w]+) + + + txn-init-time-ns-total + The total time the Producer spent initializing transactions in nanoseconds (for EOS). + kafka.producer:type=producer-metrics,client-id=([-.\w]+) + + + txn-begin-time-ns-total + The total time the Producer spent in beginTransaction in nanoseconds (for EOS). + kafka.producer:type=producer-metrics,client-id=([-.\w]+) + + + txn-send-offsets-time-ns-total + The total time the Producer spent sending offsets to transactions in nanoseconds (for EOS). + kafka.producer:type=producer-metrics,client-id=([-.\w]+) + + + txn-commit-time-ns-total + The total time the Producer spent committing transactions in nanoseconds (for EOS). + kafka.producer:type=producer-metrics,client-id=([-.\w]+) + + + txn-abort-time-ns-total + The total time the Producer spent aborting transactions in nanoseconds (for EOS). + kafka.producer:type=producer-metrics,client-id=([-.\w]+) + @@ -2120,7 +2371,7 @@

    -

    consumer monitoring

    +

    Consumer monitoring

    The following metrics are available on consumer instances. @@ -2151,6 +2402,16 @@

    < The average fraction of time the consumer's poll() is idle as opposed to waiting for the user code to process records. kafka.consumer:type=consumer-metrics,client-id=([-.\w]+) + + commited-time-ns-total + The total time the Consumer spent in committed in nanoseconds. + kafka.consumer:type=consumer-metrics,client-id=([-.\w]+) + + + commit-sync-time-ns-total + The total time the Consumer spent committing offsets in nanoseconds (for AOS). + kafka.consumer:type=consumer-metrics,client-id=([-.\w]+) + @@ -2499,6 +2760,16 @@

    6.10 KRaft
    + +

    Configuration

    + +
    Process Roles
    + +

    In KRaft mode each Kafka server can be configured as a controller, a broker, or both using the process.roles property. This property can have the following values:

    + +
      +
    • If process.roles is set to broker, the server acts as a broker.
    • +
    • If process.roles is set to controller, the server acts as a controller.
    • +
    • If process.roles is set to broker,controller, the server acts as both a broker and a controller.
    • +
    • If process.roles is not set at all, it is assumed to be in ZooKeeper mode.
    • +
    + +

    Kafka servers that act as both brokers and controllers are referred to as "combined" servers. Combined servers are simpler to operate for small use cases like a development environment. The key disadvantage is that the controller will be less isolated from the rest of the system. For example, it is not possible to roll or scale the controllers separately from the brokers in combined mode. Combined mode is not recommended in critical deployment environments.

    + + +
    Controllers
    + +

    In KRaft mode, specific Kafka servers are selected to be controllers (unlike the ZooKeeper-based mode, where any server can become the Controller). The servers selected to be controllers will participate in the metadata quorum. Each controller is either an active or a hot standby for the current active controller.

    + +

    A Kafka admin will typically select 3 or 5 servers for this role, depending on factors like cost and the number of concurrent failures your system should withstand without availability impact. A majority of the controllers must be alive in order to maintain availability. With 3 controllers, the cluster can tolerate 1 controller failure; with 5 controllers, the cluster can tolerate 2 controller failures.

    + +

    All of the servers in a Kafka cluster discover the quorum voters using the controller.quorum.voters property. This identifies the quorum controller servers that should be used. All the controllers must be enumerated. Each controller is identified with their id, host and port information. For example:

    + +
    controller.quorum.voters=id1@host1:port1,id2@host2:port2,id3@host3:port3
    + +

    If a Kafka cluster has 3 controllers named controller1, controller2 and controller3, then controller1 may have the following configuration:

    + +
    
    +process.roles=controller
    +node.id=1
    +listeners=CONTROLLER://controller1.example.com:9093
    +controller.quorum.voters=1@controller1.example.com:9093,2@controller2.example.com:9093,3@controller3.example.com:9093
    + +

    Every broker and controller must set the controller.quorum.voters property. The node ID supplied in the controller.quorum.voters property must match the corresponding id on the controller servers. For example, on controller1, node.id must be set to 1, and so forth. Each node ID must be unique across all the servers in a particular cluster. No two servers can have the same node ID regardless of their process.roles values. + +

    Storage Tool

    +

    + The kafka-storage.sh random-uuid command can be used to generate a cluster ID for your new cluster. This cluster ID must be used when formatting each server in the cluster with the kafka-storage.sh format command. + +

    This is different from how Kafka has operated in the past. Previously, Kafka would format blank storage directories automatically, and also generate a new cluster ID automatically. One reason for the change is that auto-formatting can sometimes obscure an error condition. This is particularly important for the metadata log maintained by the controller and broker servers. If a majority of the controllers were able to start with an empty log directory, a leader might be able to be elected with missing committed data.

    + +

    Debugging

    + +
    Metadata Quorum Tool
    + +

    The kafka-metadata-quorum tool can be used to describe the runtime state of the cluster metadata partition. For example, the following command displays a summary of the metadata quorum:

    + +
      > bin/kafka-metadata-quorum.sh --bootstrap-server  broker_host:port describe --status
    +ClusterId:              fMCL8kv1SWm87L_Md-I2hg
    +LeaderId:               3002
    +LeaderEpoch:            2
    +HighWatermark:          10
    +MaxFollowerLag:         0
    +MaxFollowerLagTimeMs:   -1
    +CurrentVoters:          [3000,3001,3002]
    +CurrentObservers:       [0,1,2]
    + +
    Dump Log Tool
    + +

    The kafka-dump-log tool can be used to debug the log segments and snapshots for the cluster metadata directory. The tool will scan the provided files and decode the metadata records. For example, this command decodes and prints the records in the first log segment:

    + +
      > bin/kafka-dump-log.sh --cluster-metadata-decoder --files metadata_log_dir/__cluster_metadata-0/00000000000000000000.log
    + +

    This command decodes and prints the recrods in the a cluster metadata snapshot:

    + +
      > bin/kafka-dump-log.sh --cluster-metadata-decoder --files metadata_log_dir/__cluster_metadata-0/00000000000000000100-0000000001.checkpoint
    + +
    Metadata Shell
    + +

    The kafka-metadata-shell tool can be used to interactively inspect the state of the cluster metadata partition:

    + +
    
    +  > bin/kafka-metadata-shell.sh  --snapshot metadata_log_dir/__cluster_metadata-0/00000000000000000000.log
    +>> ls /
    +brokers  local  metadataQuorum  topicIds  topics
    +>> ls /topics
    +foo
    +>> cat /topics/foo/0/data
    +{
    +  "partitionId" : 0,
    +  "topicId" : "5zoAlv-xEh9xRANKXt1Lbg",
    +  "replicas" : [ 1 ],
    +  "isr" : [ 1 ],
    +  "removingReplicas" : null,
    +  "addingReplicas" : null,
    +  "leader" : 1,
    +  "leaderEpoch" : 0,
    +  "partitionEpoch" : 0
    +}
    +>> exit
    +  
    + +

    Deploying Considerations

    + +