diff --git a/.github/workflows/docker_build_and_test.yml b/.github/workflows/docker_build_and_test.yml index 695c08672fd87..1d30aa85ea25d 100644 --- a/.github/workflows/docker_build_and_test.yml +++ b/.github/workflows/docker_build_and_test.yml @@ -23,6 +23,7 @@ on: description: Docker image type to build and test options: - "jvm" + - "native" kafka_url: description: Kafka url to be used to build the docker image required: true diff --git a/.github/workflows/docker_official_image_build_and_test.yml b/.github/workflows/docker_official_image_build_and_test.yml new file mode 100644 index 0000000000000..a315cd0e0d2ac --- /dev/null +++ b/.github/workflows/docker_official_image_build_and_test.yml @@ -0,0 +1,66 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: Docker Official Image Build Test + +on: + workflow_dispatch: + inputs: + image_type: + type: choice + description: Docker image type to build and test + options: + - "jvm" + kafka_version: + description: Kafka version for the docker official image. This should be >=3.7.0 + required: true + +jobs: + build: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - name: Set up Python 3.10 + uses: actions/setup-python@v3 + with: + python-version: "3.10" + - name: Install dependencies + run: | + python -m pip install --upgrade pip + pip install -r docker/requirements.txt + - name: Build image and run tests + working-directory: ./docker + run: | + python docker_official_image_build_test.py kafka/test -tag=test -type=${{ github.event.inputs.image_type }} -v=${{ github.event.inputs.kafka_version }} + - name: Run CVE scan + uses: aquasecurity/trivy-action@master + with: + image-ref: 'kafka/test:test' + format: 'table' + severity: 'CRITICAL,HIGH' + output: scan_report_${{ github.event.inputs.image_type }}.txt + exit-code: '1' + - name: Upload test report + if: always() + uses: actions/upload-artifact@v3 + with: + name: report_${{ github.event.inputs.image_type }}.html + path: docker/test/report_${{ github.event.inputs.image_type }}.html + - name: Upload CVE scan report + if: always() + uses: actions/upload-artifact@v3 + with: + name: scan_report_${{ github.event.inputs.image_type }}.txt + path: scan_report_${{ github.event.inputs.image_type }}.txt diff --git a/.github/workflows/docker_promote.yml b/.github/workflows/docker_promote.yml index 3449265877b8a..04872f9d59d3b 100644 --- a/.github/workflows/docker_promote.yml +++ b/.github/workflows/docker_promote.yml @@ -19,10 +19,10 @@ on: workflow_dispatch: inputs: rc_docker_image: - description: RC docker image that needs to be promoted (Example:- apache/kafka:3.6.0-rc0) + description: RC docker image that needs to be promoted (Example:- apache/kafka:3.8.0-rc0 (OR) apache/kafka-native:3.8.0-rc0) required: true promoted_docker_image: - description: Docker image name of the promoted image (Example:- apache/kafka:3.6.0) + description: Docker image name of the promoted image (Example:- apache/kafka:3.8.0 (OR) apache/kafka-native:3.8.0) required: true jobs: diff --git a/.github/workflows/docker_rc_release.yml b/.github/workflows/docker_rc_release.yml index c7082dcac910c..22dd924b51b61 100644 --- a/.github/workflows/docker_rc_release.yml +++ b/.github/workflows/docker_rc_release.yml @@ -23,8 +23,9 @@ on: description: Docker image type to be built and pushed options: - "jvm" + - "native" rc_docker_image: - description: RC docker image that needs to be built and pushed to Dockerhub (Example:- apache/kafka:3.6.0-rc0) + description: RC docker image that needs to be built and pushed to Dockerhub (Example:- apache/kafka:3.8.0-rc0 (OR) apache/kafka-native:3.8.0-rc0) required: true kafka_url: description: Kafka url to be used to build the docker image diff --git a/.github/workflows/prepare_docker_official_image_source.yml b/.github/workflows/prepare_docker_official_image_source.yml new file mode 100644 index 0000000000000..45491045836f6 --- /dev/null +++ b/.github/workflows/prepare_docker_official_image_source.yml @@ -0,0 +1,52 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: Docker Prepare Docker Official Image Source + +on: + workflow_dispatch: + inputs: + image_type: + type: choice + description: Docker image type to build and test + options: + - "jvm" + kafka_version: + description: Kafka version for the docker official image. This should be >=3.7.0 + required: true + +jobs: + build: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - name: Set up Python 3.10 + uses: actions/setup-python@v3 + with: + python-version: "3.10" + - name: Install dependencies + run: | + python -m pip install --upgrade pip + pip install -r docker/requirements.txt + - name: Build Docker Official Image Artifact + working-directory: ./docker + run: | + python prepare_docker_official_image_source.py -type=${{ github.event.inputs.image_type }} -v=${{ github.event.inputs.kafka_version }} + - name: Upload Docker Official Image Artifact + if: success() + uses: actions/upload-artifact@v4 + with: + name: ${{ github.event.inputs.kafka_version }} + path: docker/docker_official_images/${{ github.event.inputs.kafka_version }} diff --git a/.gitignore b/.gitignore index 7dfe61c38cccf..015df8ead8327 100644 --- a/.gitignore +++ b/.gitignore @@ -60,4 +60,5 @@ jmh-benchmarks/src/main/generated storage/kafka-tiered-storage/ docker/test/report_*.html +kafka.Kafka __pycache__ diff --git a/Jenkinsfile b/Jenkinsfile index 50b7f6a298e71..0a795637ff638 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -36,6 +36,12 @@ def doTest(env, target = "test") { junit '**/build/test-results/**/TEST-*.xml' } +def runTestOnDevBranch(env) { + if (!isChangeRequest(env)) { + doTest(env) + } +} + def doStreamsArchetype() { echo 'Verify that Kafka Streams archetype compiles' @@ -132,7 +138,7 @@ pipeline { } steps { doValidation() - doTest(env) + runTestOnDevBranch(env) echo 'Skipping Kafka Streams archetype test for Java 11' } } @@ -151,7 +157,7 @@ pipeline { } steps { doValidation() - doTest(env) + runTestOnDevBranch(env) echo 'Skipping Kafka Streams archetype test for Java 17' } } diff --git a/README.md b/README.md index 27ce0dc0bce64..ab7dcd7685bde 100644 --- a/README.md +++ b/README.md @@ -227,11 +227,16 @@ There are two code quality analysis tools that we regularly run, spotbugs and ch Checkstyle enforces a consistent coding style in Kafka. You can run checkstyle using: - ./gradlew checkstyleMain checkstyleTest + ./gradlew checkstyleMain checkstyleTest spotlessCheck The checkstyle warnings will be found in `reports/checkstyle/reports/main.html` and `reports/checkstyle/reports/test.html` files in the subproject build directories. They are also printed to the console. The build will fail if Checkstyle fails. +#### Spotless #### +The import order is a part of static check. please call `spotlessApply` to optimize the imports of Java codes before filing pull request : + + ./gradlew spotlessApply + #### Spotbugs #### Spotbugs uses static analysis to look for bugs in the code. You can run spotbugs using: diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 4e63ff1fb972a..e1d9e540495d5 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -208,7 +208,7 @@ fi # JMX settings if [ -z "$KAFKA_JMX_OPTS" ]; then - KAFKA_JMX_OPTS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false " + KAFKA_JMX_OPTS="-Dcom.sun.management.jmxremote=true -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false " fi # JMX port to use @@ -340,9 +340,16 @@ CLASSPATH=${CLASSPATH#:} # If Cygwin is detected, classpath is converted to Windows format. (( WINDOWS_OS_FORMAT )) && CLASSPATH=$(cygpath --path --mixed "${CLASSPATH}") -# Launch mode -if [ "x$DAEMON_MODE" = "xtrue" ]; then - nohup "$JAVA" $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS $KAFKA_LOG4J_CMD_OPTS -cp "$CLASSPATH" $KAFKA_OPTS "$@" > "$CONSOLE_OUTPUT_FILE" 2>&1 < /dev/null & +# If KAFKA_MODE=native, it will bring up Kafka in the native mode. +# It expects the Kafka executable binary to be present at $base_dir/kafka.Kafka. +# This is specifically used to run system tests on native Kafka - by bringing up Kafka in the native mode. +if [[ "x$KAFKA_MODE" == "xnative" ]] && [[ "$*" == *"kafka.Kafka"* ]]; then + exec $base_dir/kafka.Kafka start --config "$2" $KAFKA_LOG4J_CMD_OPTS $KAFKA_JMX_OPTS $KAFKA_OPTS else - exec "$JAVA" $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS $KAFKA_LOG4J_CMD_OPTS -cp "$CLASSPATH" $KAFKA_OPTS "$@" + # Launch mode + if [ "x$DAEMON_MODE" = "xtrue" ]; then + nohup "$JAVA" $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS $KAFKA_LOG4J_CMD_OPTS -cp "$CLASSPATH" $KAFKA_OPTS "$@" > "$CONSOLE_OUTPUT_FILE" 2>&1 < /dev/null & + else + exec "$JAVA" $KAFKA_HEAP_OPTS $KAFKA_JVM_PERFORMANCE_OPTS $KAFKA_GC_LOG_OPTS $KAFKA_JMX_OPTS $KAFKA_LOG4J_CMD_OPTS -cp "$CLASSPATH" $KAFKA_OPTS "$@" + fi fi diff --git a/build.gradle b/build.gradle index 031389f62c6e9..a2a6531d29a62 100644 --- a/build.gradle +++ b/build.gradle @@ -47,7 +47,9 @@ plugins { // Updating the shadow plugin version to 8.1.1 causes issue with signing and publishing the shadowed // artifacts - see https://github.com/johnrengelman/shadow/issues/901 id 'com.github.johnrengelman.shadow' version '8.1.0' apply false - id 'com.diffplug.spotless' version '6.14.0' apply false // 6.14.1 and newer require Java 11 at compile time, so we can't upgrade until AK 4.0 + // the minimum required JRE of 6.14.0+ is 11 + // refer:https://github.com/diffplug/spotless/tree/main/plugin-gradle#requirements + id 'com.diffplug.spotless' version "6.13.0" apply false } ext { @@ -176,6 +178,8 @@ allprojects { options.memberLevel = JavadocMemberLevel.PUBLIC // Document only public members/API // Turn off doclint for now, see https://blog.joda.org/2014/02/turning-off-doclint-in-jdk-8-javadoc.html for rationale options.addStringOption('Xdoclint:none', '-quiet') + // Javadoc warnings should fail the build in JDK 15+ https://bugs.openjdk.org/browse/JDK-8200363 + options.addBooleanOption('Werror', JavaVersion.current().isCompatibleWith(JavaVersion.VERSION_15)) // The URL structure was changed to include the locale after Java 8 if (JavaVersion.current().isJava11Compatible()) @@ -196,6 +200,9 @@ def determineCommitId() { } } +def spotlessApplyModules = [''] + + apply from: file('wrapper.gradle') if (repo != null) { @@ -233,7 +240,8 @@ if (repo != null) { '**/generated/**', 'clients/src/test/resources/serializedData/*', 'docker/test/fixtures/secrets/*', - 'docker/examples/fixtures/secrets/*' + 'docker/examples/fixtures/secrets/*', + 'docker/docker_official_images/.gitkeep' ]) } } else { @@ -304,10 +312,12 @@ subprojects { addParametersForTests(name, options) } - // We should only set this if Java version is < 9 (--release is recommended for >= 9), but the Scala plugin for IntelliJ sets - // `-target` incorrectly if this is unset - sourceCompatibility = minJavaVersion - targetCompatibility = minJavaVersion + java { + // We should only set this if Java version is < 9 (--release is recommended for >= 9), but the Scala plugin for IntelliJ sets + // `-target` incorrectly if this is unset + sourceCompatibility = minJavaVersion + targetCompatibility = minJavaVersion + } if (shouldPublish) { @@ -356,7 +366,7 @@ subprojects { artifact task } - artifactId = archivesBaseName + artifactId = base.archivesName.get() pom { name = 'Apache Kafka' url = 'https://kafka.apache.org' @@ -609,6 +619,8 @@ subprojects { task docsJar(dependsOn: javadocJar) + test.dependsOn('javadoc') + task systemTestLibs(dependsOn: jar) if (!sourceSets.test.allSource.isEmpty()) { @@ -741,8 +753,8 @@ subprojects { } test.dependsOn('spotbugsMain') - tasks.withType(com.github.spotbugs.snom.SpotBugsTask) { - reports { + tasks.withType(com.github.spotbugs.snom.SpotBugsTask).configureEach { + reports.configure { // Continue supporting `xmlFindBugsReport` for compatibility xml.enabled(project.hasProperty('xmlSpotBugsReport') || project.hasProperty('xmlFindBugsReport')) html.enabled(!project.hasProperty('xmlSpotBugsReport') && !project.hasProperty('xmlFindBugsReport')) @@ -786,6 +798,16 @@ subprojects { skipProjects = [ ":jmh-benchmarks", ":trogdor" ] skipConfigurations = [ "zinc" ] } + + if (project.name in spotlessApplyModules) { + apply plugin: 'com.diffplug.spotless' + spotless { + java { + importOrder('kafka', 'org.apache.kafka', 'com', 'net', 'org', 'java', 'javax', '', '\\#') + removeUnusedImports() + } + } + } } gradle.taskGraph.whenReady { taskGraph -> @@ -850,7 +872,9 @@ tasks.create(name: "jarConnect", dependsOn: connectPkgs.collect { it + ":jar" }) tasks.create(name: "testConnect", dependsOn: connectPkgs.collect { it + ":test" }) {} project(':server') { - archivesBaseName = "kafka-server" + base { + archivesName = "kafka-server" + } dependencies { implementation project(':clients') @@ -922,7 +946,10 @@ project(':core') { } if (userEnableTestCoverage) apply plugin: "org.scoverage" - archivesBaseName = "kafka_${versions.baseScala}" + + base { + archivesName = "kafka_${versions.baseScala}" + } configurations { generator @@ -1250,7 +1277,9 @@ project(':core') { } project(':metadata') { - archivesBaseName = "kafka-metadata" + base { + archivesName = "kafka-metadata" + } configurations { generator @@ -1317,7 +1346,9 @@ project(':metadata') { } project(':group-coordinator') { - archivesBaseName = "kafka-group-coordinator" + base { + archivesName = "kafka-group-coordinator" + } configurations { generator @@ -1384,7 +1415,9 @@ project(':group-coordinator') { } project(':transaction-coordinator') { - archivesBaseName = "kafka-transaction-coordinator" + base { + archivesName = "kafka-transaction-coordinator" + } sourceSets { main { @@ -1405,7 +1438,9 @@ project(':transaction-coordinator') { } project(':examples') { - archivesBaseName = "kafka-examples" + base { + archivesName = "kafka-examples" + } dependencies { implementation project(':clients') @@ -1435,7 +1470,9 @@ project(':generator') { } project(':clients') { - archivesBaseName = "kafka-clients" + base { + archivesName = "kafka-clients" + } configurations { generator @@ -1610,7 +1647,9 @@ project(':clients') { } project(':raft') { - archivesBaseName = "kafka-raft" + base { + archivesName = "kafka-raft" + } configurations { generator @@ -1706,7 +1745,9 @@ project(':raft') { } project(':server-common') { - archivesBaseName = "kafka-server-common" + base { + archivesName = "kafka-server-common" + } dependencies { api project(':clients') @@ -1764,7 +1805,9 @@ project(':server-common') { } project(':storage:storage-api') { - archivesBaseName = "kafka-storage-api" + base { + archivesName = "kafka-storage-api" + } dependencies { implementation project(':clients') @@ -1832,7 +1875,9 @@ project(':storage:storage-api') { } project(':storage') { - archivesBaseName = "kafka-storage" + base { + archivesName = "kafka-storage" + } configurations { generator @@ -1954,7 +1999,9 @@ project(':storage') { } project(':tools:tools-api') { - archivesBaseName = "kafka-tools-api" + base { + archivesName = "kafka-tools-api" + } dependencies { implementation project(':clients') @@ -2009,7 +2056,10 @@ project(':tools:tools-api') { } project(':tools') { - archivesBaseName = "kafka-tools" + base { + archivesName = "kafka-tools" + } + dependencies { implementation project(':clients') implementation project(':storage') @@ -2073,7 +2123,9 @@ project(':tools') { } project(':trogdor') { - archivesBaseName = "trogdor" + base { + archivesName = "trogdor" + } dependencies { implementation project(':clients') @@ -2123,7 +2175,9 @@ project(':trogdor') { } project(':shell') { - archivesBaseName = "kafka-shell" + base { + archivesName = "kafka-shell" + } dependencies { implementation libs.argparse4j @@ -2173,7 +2227,10 @@ project(':shell') { } project(':streams') { - archivesBaseName = "kafka-streams" + base { + archivesName = "kafka-streams" + } + ext.buildStreamsVersionFileName = "kafka-streams-version.properties" configurations { @@ -2335,7 +2392,11 @@ project(':streams') { project(':streams:streams-scala') { apply plugin: 'scala' - archivesBaseName = "kafka-streams-scala_${versions.baseScala}" + + base { + archivesName = "kafka-streams-scala_${versions.baseScala}" + } + dependencies { api project(':streams') @@ -2397,7 +2458,9 @@ project(':streams:streams-scala') { } project(':streams:test-utils') { - archivesBaseName = "kafka-streams-test-utils" + base { + archivesName = "kafka-streams-test-utils" + } dependencies { api project(':streams') @@ -2432,7 +2495,9 @@ project(':streams:test-utils') { } project(':streams:examples') { - archivesBaseName = "kafka-streams-examples" + base { + archivesName = "kafka-streams-examples" + } dependencies { // this dependency should be removed after we unify data API @@ -2469,7 +2534,9 @@ project(':streams:examples') { } project(':streams:upgrade-system-tests-0100') { - archivesBaseName = "kafka-streams-upgrade-system-tests-0100" + base { + archivesName = "kafka-streams-upgrade-system-tests-0100" + } dependencies { testImplementation(libs.kafkaStreams_0100) { @@ -2485,7 +2552,9 @@ project(':streams:upgrade-system-tests-0100') { } project(':streams:upgrade-system-tests-0101') { - archivesBaseName = "kafka-streams-upgrade-system-tests-0101" + base { + archivesName = "kafka-streams-upgrade-system-tests-0101" + } dependencies { testImplementation(libs.kafkaStreams_0101) { @@ -2501,7 +2570,9 @@ project(':streams:upgrade-system-tests-0101') { } project(':streams:upgrade-system-tests-0102') { - archivesBaseName = "kafka-streams-upgrade-system-tests-0102" + base { + archivesName = "kafka-streams-upgrade-system-tests-0102" + } dependencies { testImplementation libs.kafkaStreams_0102 @@ -2514,7 +2585,9 @@ project(':streams:upgrade-system-tests-0102') { } project(':streams:upgrade-system-tests-0110') { - archivesBaseName = "kafka-streams-upgrade-system-tests-0110" + base{ + archivesName = "kafka-streams-upgrade-system-tests-0110" + } dependencies { testImplementation libs.kafkaStreams_0110 @@ -2527,7 +2600,9 @@ project(':streams:upgrade-system-tests-0110') { } project(':streams:upgrade-system-tests-10') { - archivesBaseName = "kafka-streams-upgrade-system-tests-10" + base { + archivesName = "kafka-streams-upgrade-system-tests-10" + } dependencies { testImplementation libs.kafkaStreams_10 @@ -2540,7 +2615,9 @@ project(':streams:upgrade-system-tests-10') { } project(':streams:upgrade-system-tests-11') { - archivesBaseName = "kafka-streams-upgrade-system-tests-11" + base { + archivesName = "kafka-streams-upgrade-system-tests-11" + } dependencies { testImplementation libs.kafkaStreams_11 @@ -2553,7 +2630,9 @@ project(':streams:upgrade-system-tests-11') { } project(':streams:upgrade-system-tests-20') { - archivesBaseName = "kafka-streams-upgrade-system-tests-20" + base { + archivesName = "kafka-streams-upgrade-system-tests-20" + } dependencies { testImplementation libs.kafkaStreams_20 @@ -2566,7 +2645,9 @@ project(':streams:upgrade-system-tests-20') { } project(':streams:upgrade-system-tests-21') { - archivesBaseName = "kafka-streams-upgrade-system-tests-21" + base { + archivesName = "kafka-streams-upgrade-system-tests-21" + } dependencies { testImplementation libs.kafkaStreams_21 @@ -2579,7 +2660,9 @@ project(':streams:upgrade-system-tests-21') { } project(':streams:upgrade-system-tests-22') { - archivesBaseName = "kafka-streams-upgrade-system-tests-22" + base { + archivesName = "kafka-streams-upgrade-system-tests-22" + } dependencies { testImplementation libs.kafkaStreams_22 @@ -2592,7 +2675,9 @@ project(':streams:upgrade-system-tests-22') { } project(':streams:upgrade-system-tests-23') { - archivesBaseName = "kafka-streams-upgrade-system-tests-23" + base { + archivesName = "kafka-streams-upgrade-system-tests-23" + } dependencies { testImplementation libs.kafkaStreams_23 @@ -2605,7 +2690,9 @@ project(':streams:upgrade-system-tests-23') { } project(':streams:upgrade-system-tests-24') { - archivesBaseName = "kafka-streams-upgrade-system-tests-24" + base { + archivesName = "kafka-streams-upgrade-system-tests-24" + } dependencies { testImplementation libs.kafkaStreams_24 @@ -2618,7 +2705,9 @@ project(':streams:upgrade-system-tests-24') { } project(':streams:upgrade-system-tests-25') { - archivesBaseName = "kafka-streams-upgrade-system-tests-25" + base { + archivesName = "kafka-streams-upgrade-system-tests-25" + } dependencies { testImplementation libs.kafkaStreams_25 @@ -2631,7 +2720,9 @@ project(':streams:upgrade-system-tests-25') { } project(':streams:upgrade-system-tests-26') { - archivesBaseName = "kafka-streams-upgrade-system-tests-26" + base { + archivesName = "kafka-streams-upgrade-system-tests-26" + } dependencies { testImplementation libs.kafkaStreams_26 @@ -2644,7 +2735,9 @@ project(':streams:upgrade-system-tests-26') { } project(':streams:upgrade-system-tests-27') { - archivesBaseName = "kafka-streams-upgrade-system-tests-27" + base { + archivesName = "kafka-streams-upgrade-system-tests-27" + } dependencies { testImplementation libs.kafkaStreams_27 @@ -2657,7 +2750,9 @@ project(':streams:upgrade-system-tests-27') { } project(':streams:upgrade-system-tests-28') { - archivesBaseName = "kafka-streams-upgrade-system-tests-28" + base { + archivesName = "kafka-streams-upgrade-system-tests-28" + } dependencies { testImplementation libs.kafkaStreams_28 @@ -2670,7 +2765,9 @@ project(':streams:upgrade-system-tests-28') { } project(':streams:upgrade-system-tests-30') { - archivesBaseName = "kafka-streams-upgrade-system-tests-30" + base { + archivesName = "kafka-streams-upgrade-system-tests-30" + } dependencies { testImplementation libs.kafkaStreams_30 @@ -2683,7 +2780,9 @@ project(':streams:upgrade-system-tests-30') { } project(':streams:upgrade-system-tests-31') { - archivesBaseName = "kafka-streams-upgrade-system-tests-31" + base { + archivesName = "kafka-streams-upgrade-system-tests-31" + } dependencies { testImplementation libs.kafkaStreams_31 @@ -2696,7 +2795,9 @@ project(':streams:upgrade-system-tests-31') { } project(':streams:upgrade-system-tests-32') { - archivesBaseName = "kafka-streams-upgrade-system-tests-32" + base { + archivesName = "kafka-streams-upgrade-system-tests-32" + } dependencies { testImplementation libs.kafkaStreams_32 @@ -2709,7 +2810,9 @@ project(':streams:upgrade-system-tests-32') { } project(':streams:upgrade-system-tests-33') { - archivesBaseName = "kafka-streams-upgrade-system-tests-33" + base { + archivesName = "kafka-streams-upgrade-system-tests-33" + } dependencies { testImplementation libs.kafkaStreams_33 @@ -2722,7 +2825,9 @@ project(':streams:upgrade-system-tests-33') { } project(':streams:upgrade-system-tests-34') { - archivesBaseName = "kafka-streams-upgrade-system-tests-34" + base { + archivesName = "kafka-streams-upgrade-system-tests-34" + } dependencies { testImplementation libs.kafkaStreams_34 @@ -2735,7 +2840,9 @@ project(':streams:upgrade-system-tests-34') { } project(':streams:upgrade-system-tests-35') { - archivesBaseName = "kafka-streams-upgrade-system-tests-35" + base { + archivesName = "kafka-streams-upgrade-system-tests-35" + } dependencies { testImplementation libs.kafkaStreams_35 @@ -2748,7 +2855,9 @@ project(':streams:upgrade-system-tests-35') { } project(':streams:upgrade-system-tests-36') { - archivesBaseName = "kafka-streams-upgrade-system-tests-36" + base { + archivesName = "kafka-streams-upgrade-system-tests-36" + } dependencies { testImplementation libs.kafkaStreams_36 @@ -2761,7 +2870,9 @@ project(':streams:upgrade-system-tests-36') { } project(':streams:upgrade-system-tests-37') { - archivesBaseName = "kafka-streams-upgrade-system-tests-37" + base { + archivesName = "kafka-streams-upgrade-system-tests-37" + } dependencies { testImplementation libs.kafkaStreams_37 @@ -2846,7 +2957,9 @@ project(':jmh-benchmarks') { } project(':log4j-appender') { - archivesBaseName = "kafka-log4j-appender" + base { + archivesName = "kafka-log4j-appender" + } dependencies { implementation project(':clients') @@ -2865,7 +2978,9 @@ project(':log4j-appender') { } project(':connect:api') { - archivesBaseName = "connect-api" + base { + archivesName = "connect-api" + } dependencies { api project(':clients') @@ -2900,7 +3015,9 @@ project(':connect:api') { } project(':connect:transforms') { - archivesBaseName = "connect-transforms" + base { + archivesName = "connect-transforms" + } dependencies { api project(':connect:api') @@ -2936,7 +3053,9 @@ project(':connect:transforms') { } project(':connect:json') { - archivesBaseName = "connect-json" + base { + archivesName = "connect-json" + } dependencies { api project(':connect:api') @@ -2980,7 +3099,9 @@ project(':connect:runtime') { swagger } - archivesBaseName = "connect-runtime" + base { + archivesName = "connect-runtime" + } dependencies { // connect-runtime is used in tests, use `api` for modules below for backwards compatibility even though @@ -3122,7 +3243,9 @@ project(':connect:runtime') { } project(':connect:file') { - archivesBaseName = "connect-file" + base { + archivesName = "connect-file" + } dependencies { implementation project(':connect:api') @@ -3162,7 +3285,9 @@ project(':connect:file') { } project(':connect:basic-auth-extension') { - archivesBaseName = "connect-basic-auth-extension" + base { + archivesName = "connect-basic-auth-extension" + } dependencies { implementation project(':connect:api') @@ -3202,7 +3327,9 @@ project(':connect:basic-auth-extension') { } project(':connect:mirror') { - archivesBaseName = "connect-mirror" + base { + archivesName = "connect-mirror" + } dependencies { implementation project(':connect:api') @@ -3290,7 +3417,9 @@ project(':connect:mirror') { } project(':connect:mirror-client') { - archivesBaseName = "connect-mirror-client" + base { + archivesName = "connect-mirror-client" + } dependencies { implementation project(':clients') @@ -3325,7 +3454,9 @@ project(':connect:mirror-client') { } project(':connect:test-plugins') { - archivesBaseName = "connect-test-plugins" + base { + archivesName = "connect-test-plugins" + } dependencies { api project(':connect:api') diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index aff659638928b..61eb7e4b245fd 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -82,6 +82,8 @@ + + diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml index ed6c53a322ba3..6724ea9bf3ba0 100644 --- a/checkstyle/import-control-core.xml +++ b/checkstyle/import-control-core.xml @@ -130,5 +130,6 @@ + diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index ed0f34e4fffad..ab6177961f51e 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -211,6 +211,11 @@ + + + + + diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index a8bc6351db3fa..fc6995dadfe7e 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -43,6 +43,8 @@ + @@ -99,7 +101,7 @@ files="(AbstractRequest|AbstractResponse|KerberosLogin|WorkerSinkTaskTest|TransactionManagerTest|SenderTest|KafkaAdminClient|ConsumerCoordinatorTest|KafkaAdminClientTest|KafkaRaftClientTest).java"/> + files="(ConsumerCoordinator|BufferPool|MetricName|Node|ConfigDef|RecordBatch|SslFactory|SslTransportLayer|MetadataResponse|KerberosLogin|Selector|Sender|Serdes|TokenInformation|Agent|PluginUtils|MiniTrogdorCluster|TasksRequest|KafkaProducer|AbstractStickyAssignor|KafkaRaftClient|Authorizer|FetchSessionHandler|RecordAccumulator|Shell).java"/> @@ -143,7 +145,7 @@ + files="(DistributedHerder|DistributedConfig|KafkaConfigBackingStore|IncrementalCooperativeAssignor).java"/> + files="(JsonConverter|ConnectHeaders).java"/> + files="(KafkaConfigBackingStore|ConnectMetricsRegistry).java"/> @@ -347,7 +349,7 @@ + files="(GroupMetadataManager|GroupMetadataManagerTest).java"/> offsetsForTimes(Map entry.getValue().buildOffsetAndTimestamp())); + Map.Entry::getKey, + entry -> entry.getValue().buildOffsetAndTimestamp())); + } catch (TimeoutException e) { + throw new TimeoutException("Failed to get offsets by times in " + timeout.toMillis() + "ms"); + } } finally { release(); } @@ -1136,12 +1140,16 @@ private Map beginningOrEndOffset(Collection offsetAndTimestampMap; - offsetAndTimestampMap = applicationEventHandler.addAndGet(listOffsetsEvent); - return offsetAndTimestampMap.entrySet() + try { + offsetAndTimestampMap = applicationEventHandler.addAndGet(listOffsetsEvent); + return offsetAndTimestampMap.entrySet() .stream() .collect(Collectors.toMap( - Map.Entry::getKey, - entry -> entry.getValue().offset())); + Map.Entry::getKey, + entry -> entry.getValue().offset())); + } catch (TimeoutException e) { + throw new TimeoutException("Failed to get offsets by times in " + timeout.toMillis() + "ms"); + } } finally { release(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java index 8959345bffdcf..5244af9c827d5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.errors.RecordDeserializationException; +import org.apache.kafka.common.errors.RecordDeserializationException.DeserializationExceptionOrigin; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeaders; @@ -311,25 +312,39 @@ ConsumerRecord parseRecord(Deserializers deserializers, Optional leaderEpoch, TimestampType timestampType, Record record) { + ByteBuffer keyBytes = record.key(); + ByteBuffer valueBytes = record.value(); + Headers headers = new RecordHeaders(record.headers()); + K key; + V value; try { - long offset = record.offset(); - long timestamp = record.timestamp(); - Headers headers = new RecordHeaders(record.headers()); - ByteBuffer keyBytes = record.key(); - K key = keyBytes == null ? null : deserializers.keyDeserializer.deserialize(partition.topic(), headers, keyBytes); - ByteBuffer valueBytes = record.value(); - V value = valueBytes == null ? null : deserializers.valueDeserializer.deserialize(partition.topic(), headers, valueBytes); - return new ConsumerRecord<>(partition.topic(), partition.partition(), offset, - timestamp, timestampType, - keyBytes == null ? ConsumerRecord.NULL_SIZE : keyBytes.remaining(), - valueBytes == null ? ConsumerRecord.NULL_SIZE : valueBytes.remaining(), - key, value, headers, leaderEpoch); + key = keyBytes == null ? null : deserializers.keyDeserializer.deserialize(partition.topic(), headers, keyBytes); } catch (RuntimeException e) { - log.error("Deserializers with error: {}", deserializers); - throw new RecordDeserializationException(partition, record.offset(), - "Error deserializing key/value for partition " + partition + - " at offset " + record.offset() + ". If needed, please seek past the record to continue consumption.", e); + log.error("Key Deserializers with error: {}", deserializers); + throw newRecordDeserializationException(DeserializationExceptionOrigin.KEY, partition, timestampType, record, e, headers); } + try { + value = valueBytes == null ? null : deserializers.valueDeserializer.deserialize(partition.topic(), headers, valueBytes); + } catch (RuntimeException e) { + log.error("Value Deserializers with error: {}", deserializers); + throw newRecordDeserializationException(DeserializationExceptionOrigin.VALUE, partition, timestampType, record, e, headers); + } + return new ConsumerRecord<>(partition.topic(), partition.partition(), record.offset(), + record.timestamp(), timestampType, + keyBytes == null ? ConsumerRecord.NULL_SIZE : keyBytes.remaining(), + valueBytes == null ? ConsumerRecord.NULL_SIZE : valueBytes.remaining(), + key, value, headers, leaderEpoch); + } + + private static RecordDeserializationException newRecordDeserializationException(DeserializationExceptionOrigin origin, + TopicPartition partition, + TimestampType timestampType, + Record record, + RuntimeException e, + Headers headers) { + return new RecordDeserializationException(origin, partition, record.offset(), record.timestamp(), timestampType, record.key(), record.value(), headers, + "Error deserializing " + origin.name() + " for partition " + partition + " at offset " + record.offset() + + ". If needed, please seek past the record to continue consumption.", e); } private Optional maybeLeaderEpoch(int leaderEpoch) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java index f65a0642a4300..d31d412c65503 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java @@ -45,6 +45,7 @@ import java.util.TreeSet; import java.util.stream.Collectors; + /** *

Manages the request creation and response handling for the heartbeat. The module creates a * {@link ConsumerGroupHeartbeatRequest} using the state stored in the {@link MembershipManager} and enqueue it to @@ -208,7 +209,11 @@ public NetworkClientDelegate.PollResult poll(long currentTimeMs) { return new NetworkClientDelegate.PollResult(heartbeatRequestState.heartbeatIntervalMs, Collections.singletonList(leaveHeartbeat)); } - boolean heartbeatNow = membershipManager.shouldHeartbeatNow() && !heartbeatRequestState.requestInFlight(); + // Case 1: The member is leaving + boolean heartbeatNow = membershipManager.state() == MemberState.LEAVING || + // Case 2: The member state indicates it should send a heartbeat without waiting for the interval, and there is no heartbeat request currently in-flight + (membershipManager.shouldHeartbeatNow() && !heartbeatRequestState.requestInFlight()); + if (!heartbeatRequestState.canSendRequest(currentTimeMs) && !heartbeatNow) { return new NetworkClientDelegate.PollResult(heartbeatRequestState.timeToNextHeartbeatMs(currentTimeMs)); } @@ -383,7 +388,15 @@ private void onErrorResponse(final ConsumerGroupHeartbeatResponse response, case UNRELEASED_INSTANCE_ID: logger.error("GroupHeartbeatRequest failed due to unreleased instance id {}: {}", membershipManager.groupInstanceId().orElse("null"), errorMessage); - handleFatalFailure(Errors.UNRELEASED_INSTANCE_ID.exception(errorMessage)); + handleFatalFailure(error.exception(errorMessage)); + break; + + case FENCED_INSTANCE_ID: + logger.error("GroupHeartbeatRequest failed due to fenced instance id {}: {}. " + + "This is expected in the case that the member was removed from the group " + + "by an admin client, and another member joined using the same group instance id.", + membershipManager.groupInstanceId().orElse("null"), errorMessage); + handleFatalFailure(error.exception(errorMessage)); break; case INVALID_REQUEST: 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 84b77ef5f40db..820adbdb5fbfa 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -385,12 +385,13 @@ public boolean equals(Object o) { Objects.equals(internalTopics, cluster.internalTopics) && Objects.equals(controller, cluster.controller) && Objects.equals(partitionsByTopicPartition, cluster.partitionsByTopicPartition) && - Objects.equals(clusterResource, cluster.clusterResource); + Objects.equals(clusterResource, cluster.clusterResource) && + Objects.equals(topicIds, cluster.topicIds); } @Override public int hashCode() { return Objects.hash(isBootstrapConfigured, nodes, unauthorizedTopics, invalidTopics, internalTopics, controller, - partitionsByTopicPartition, clusterResource); + partitionsByTopicPartition, clusterResource, topicIds); } } diff --git a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java index 29bedff0cf19b..0577b4200a3bc 100644 --- a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java +++ b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java @@ -16,6 +16,9 @@ */ package org.apache.kafka.common; +import java.util.Arrays; +import java.util.Objects; + /** * This is used to describe per-partition state in the MetadataResponse. */ @@ -88,6 +91,29 @@ public Node[] offlineReplicas() { return offlineReplicas; } + @Override + public int hashCode() { + return Objects.hash(topic, partition, leader, Arrays.hashCode(replicas), + Arrays.hashCode(inSyncReplicas), Arrays.hashCode(offlineReplicas)); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + PartitionInfo other = (PartitionInfo) obj; + return Objects.equals(topic, other.topic) && + partition == other.partition && + Objects.equals(leader, other.leader) && + Objects.deepEquals(replicas, other.replicas) && + Objects.deepEquals(inSyncReplicas, other.inSyncReplicas) && + Objects.deepEquals(offlineReplicas, other.offlineReplicas); + } + @Override public String toString() { return String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s, offlineReplicas = %s)", diff --git a/clients/src/main/java/org/apache/kafka/common/ShareGroupState.java b/clients/src/main/java/org/apache/kafka/common/ShareGroupState.java new file mode 100644 index 0000000000000..716421f3dea2a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/ShareGroupState.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.common; + +import java.util.Arrays; +import java.util.Locale; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * The share group state. + */ +public enum ShareGroupState { + UNKNOWN("Unknown"), + STABLE("Stable"), + DEAD("Dead"), + EMPTY("Empty"); + + private final static Map NAME_TO_ENUM = Arrays.stream(values()) + .collect(Collectors.toMap(state -> state.name.toUpperCase(Locale.ROOT), Function.identity())); + + private final String name; + + ShareGroupState(String name) { + this.name = name; + } + + /** + * Case-insensitive share group state lookup by string name. + */ + public static ShareGroupState parse(String name) { + ShareGroupState state = NAME_TO_ENUM.get(name.toUpperCase(Locale.ROOT)); + return state == null ? UNKNOWN : state; + } + + @Override + public String toString() { + return name; + } +} 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 791a4602cf0f1..767fa9aca015f 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 @@ -17,6 +17,9 @@ package org.apache.kafka.common.config.internals; import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.config.SslClientAuth; +import org.apache.kafka.common.security.auth.KafkaPrincipalBuilder; +import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder; import java.util.Collections; import java.util.List; @@ -30,39 +33,19 @@ public class BrokerSecurityConfigs { public static final String PRINCIPAL_BUILDER_CLASS_CONFIG = "principal.builder.class"; - public static final String SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG = "sasl.kerberos.principal.to.local.rules"; - public static final String SSL_CLIENT_AUTH_CONFIG = "ssl.client.auth"; - public static final String SASL_ENABLED_MECHANISMS_CONFIG = "sasl.enabled.mechanisms"; - 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 long DEFAULT_CONNECTIONS_MAX_REAUTH_MS = 0L; - 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 SSL_ALLOW_DN_CHANGES_CONFIG = "ssl.allow.dn.changes"; - public static final boolean DEFAULT_SSL_ALLOW_DN_CHANGES_VALUE = false; - public static final String SSL_ALLOW_SAN_CHANGES_CONFIG = "ssl.allow.san.changes"; - public static final boolean DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE = false; - - 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 " + - "authorization. If no principal builder is defined, the default behavior depends " + - "on the security protocol in use. For SSL authentication, the principal will be derived using the " + - "rules defined by " + SSL_PRINCIPAL_MAPPING_RULES_CONFIG + " applied on the distinguished " + - "name from the client certificate if one is provided; otherwise, if client authentication is not required, " + - "the principal name will be ANONYMOUS. For SASL authentication, the principal will be derived using the " + - "rules defined by " + SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG + " if GSSAPI is in use, " + - "and the SASL authentication ID for other mechanisms. For PLAINTEXT, the principal will be ANONYMOUS."; + public static final String SSL_PRINCIPAL_MAPPING_RULES_CONFIG = "ssl.principal.mapping.rules"; + public static final String DEFAULT_SSL_PRINCIPAL_MAPPING_RULES = "DEFAULT"; public static final String SSL_PRINCIPAL_MAPPING_RULES_DOC = "A list of rules for mapping from distinguished name" + " from the client certificate to short name. The rules are evaluated in order and the first rule that matches" + " a principal name is used to map it to a short name. Any later rules in the list are ignored. By default," + " distinguished name of the X.500 certificate will be the principal. For more details on the format please" + " see security authorization and acls. Note that this configuration is ignored" + " if an extension of KafkaPrincipalBuilder is provided by the " + PRINCIPAL_BUILDER_CLASS_CONFIG + "" + - " configuration."; - public static final String DEFAULT_SSL_PRINCIPAL_MAPPING_RULES = "DEFAULT"; + " configuration."; + public static final String SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG = "sasl.kerberos.principal.to.local.rules"; + public static final List DEFAULT_SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES = Collections.singletonList(DEFAULT_SSL_PRINCIPAL_MAPPING_RULES); public static final String SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DOC = "A list of rules for mapping from principal " + "names to short names (typically operating system usernames). The rules are evaluated in order and the " + "first rule that matches a principal name is used to map it to a short name. Any later rules in the list are " + @@ -70,8 +53,20 @@ public class BrokerSecurityConfigs { "to {username}. For more details on the format please see " + "security authorization and acls. Note that this configuration is ignored if an extension of " + "KafkaPrincipalBuilder is provided by the " + PRINCIPAL_BUILDER_CLASS_CONFIG + " configuration."; - public static final List DEFAULT_SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES = Collections.singletonList("DEFAULT"); + public static final Class PRINCIPAL_BUILDER_CLASS_DEFAULT = DefaultKafkaPrincipalBuilder.class; + 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 " + + "authorization. If no principal builder is defined, the default behavior depends " + + "on the security protocol in use. For SSL authentication, the principal will be derived using the " + + "rules defined by " + SSL_PRINCIPAL_MAPPING_RULES_CONFIG + " applied on the distinguished " + + "name from the client certificate if one is provided; otherwise, if client authentication is not required, " + + "the principal name will be ANONYMOUS. For SASL authentication, the principal will be derived using the " + + "rules defined by " + SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG + " if GSSAPI is in use, " + + "and the SASL authentication ID for other mechanisms. For PLAINTEXT, the principal will be ANONYMOUS."; + + public static final String SSL_CLIENT_AUTH_CONFIG = "ssl.client.auth"; + public static final String SSL_CLIENT_AUTH_DEFAULT = SslClientAuth.NONE.toString(); public static final String SSL_CLIENT_AUTH_DOC = "Configures kafka broker to request client authentication." + " The following settings are common: " + "

    " @@ -81,29 +76,39 @@ public class BrokerSecurityConfigs { + "
  • ssl.client.auth=none This means client authentication is not needed." + "
"; + public static final String SASL_ENABLED_MECHANISMS_CONFIG = "sasl.enabled.mechanisms"; + public static final List DEFAULT_SASL_ENABLED_MECHANISMS = Collections.singletonList(SaslConfigs.GSSAPI_MECHANISM); public static final String SASL_ENABLED_MECHANISMS_DOC = "The list of SASL mechanisms enabled in the Kafka server. " + "The list may contain any mechanism for which a security provider is available. " + "Only GSSAPI is enabled by default."; - public static final List DEFAULT_SASL_ENABLED_MECHANISMS = Collections.singletonList(SaslConfigs.GSSAPI_MECHANISM); + public static final String SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG = "sasl.server.callback.handler.class"; public static final String SASL_SERVER_CALLBACK_HANDLER_CLASS_DOC = "The fully qualified name of a SASL server callback handler " + "class that implements the AuthenticateCallbackHandler interface. Server callback handlers must be prefixed with " + "listener prefix and SASL mechanism name in lower-case. For example, " + "listener.name.sasl_ssl.plain.sasl.server.callback.handler.class=com.example.CustomPlainCallbackHandler."; + public static final String CONNECTIONS_MAX_REAUTH_MS_CONFIG = "connections.max.reauth.ms"; + public static final long DEFAULT_CONNECTIONS_MAX_REAUTH_MS = 0L; public static final String CONNECTIONS_MAX_REAUTH_MS_DOC = "When explicitly set to a positive number (the default is 0, not a positive number), " + "a session lifetime that will not exceed the configured value will be communicated to v2.2.0 or later clients when they authenticate. " + "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_CONFIG = "sasl.server.max.receive.size"; + public static final int DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE = 524288; 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."; + public static final String SSL_ALLOW_DN_CHANGES_CONFIG = "ssl.allow.dn.changes"; + public static final boolean DEFAULT_SSL_ALLOW_DN_CHANGES_VALUE = false; public static final String SSL_ALLOW_DN_CHANGES_DOC = "Indicates whether changes to the certificate distinguished name should be allowed during" + " a dynamic reconfiguration of certificates or not."; + public static final String SSL_ALLOW_SAN_CHANGES_CONFIG = "ssl.allow.san.changes"; + public static final boolean DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE = false; public static final String SSL_ALLOW_SAN_CHANGES_DOC = "Indicates whether changes to the certificate subject alternative names should be allowed during " + "a dynamic reconfiguration of certificates or not."; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/FencedStateEpochException.java b/clients/src/main/java/org/apache/kafka/common/errors/FencedStateEpochException.java new file mode 100644 index 0000000000000..1e74bba199402 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/FencedStateEpochException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +/** + * Thrown when the share coordinator rejected the request because the share-group state epoch did not match. + */ +public class FencedStateEpochException extends ApiException { + private static final long serialVersionUID = 1L; + + public FencedStateEpochException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidRecordStateException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRecordStateException.java new file mode 100644 index 0000000000000..ae0fef5edeaef --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRecordStateException.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +/** + * Thrown when the acknowledgement of delivery of a record could not be completed because the record + * state is invalid. + */ +public class InvalidRecordStateException extends ApiException { + + private static final long serialVersionUID = 1L; + + public InvalidRecordStateException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidShareSessionEpochException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidShareSessionEpochException.java new file mode 100644 index 0000000000000..e261d8b7a8e88 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidShareSessionEpochException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +/** + * Thrown when the share session epoch is invalid. + */ +public class InvalidShareSessionEpochException extends RetriableException { + private static final long serialVersionUID = 1L; + + public InvalidShareSessionEpochException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RecordDeserializationException.java b/clients/src/main/java/org/apache/kafka/common/errors/RecordDeserializationException.java index a15df6c7ff52c..aee57c47d28de 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/RecordDeserializationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/RecordDeserializationException.java @@ -16,7 +16,12 @@ */ package org.apache.kafka.common.errors; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.record.TimestampType; + +import java.nio.ByteBuffer; /** * This exception is raised for any error that occurs while deserializing records received by the consumer using @@ -24,14 +29,61 @@ */ public class RecordDeserializationException extends SerializationException { - private static final long serialVersionUID = 1L; + private static final long serialVersionUID = 2L; + + public enum DeserializationExceptionOrigin { + KEY, + VALUE + } + + private final DeserializationExceptionOrigin origin; private final TopicPartition partition; private final long offset; + private final TimestampType timestampType; + private final long timestamp; + private final ByteBuffer keyBuffer; + private final ByteBuffer valueBuffer; + private final Headers headers; - public RecordDeserializationException(TopicPartition partition, long offset, String message, Throwable cause) { + @Deprecated + public RecordDeserializationException(TopicPartition partition, + long offset, + String message, + Throwable cause) { super(message, cause); + this.origin = null; this.partition = partition; this.offset = offset; + this.timestampType = TimestampType.NO_TIMESTAMP_TYPE; + this.timestamp = ConsumerRecord.NO_TIMESTAMP; + this.keyBuffer = null; + this.valueBuffer = null; + this.headers = null; + } + + public RecordDeserializationException(DeserializationExceptionOrigin origin, + TopicPartition partition, + long offset, + long timestamp, + TimestampType timestampType, + ByteBuffer keyBuffer, + ByteBuffer valueBuffer, + Headers headers, + String message, + Throwable cause) { + super(message, cause); + this.origin = origin; + this.offset = offset; + this.timestampType = timestampType; + this.timestamp = timestamp; + this.partition = partition; + this.keyBuffer = keyBuffer; + this.valueBuffer = valueBuffer; + this.headers = headers; + } + + public DeserializationExceptionOrigin origin() { + return origin; } public TopicPartition topicPartition() { @@ -41,4 +93,24 @@ public TopicPartition topicPartition() { public long offset() { return offset; } + + public TimestampType timestampType() { + return timestampType; + } + + public long timestamp() { + return timestamp; + } + + public ByteBuffer keyBuffer() { + return keyBuffer; + } + + public ByteBuffer valueBuffer() { + return valueBuffer; + } + + public Headers headers() { + return headers; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ShareSessionNotFoundException.java b/clients/src/main/java/org/apache/kafka/common/errors/ShareSessionNotFoundException.java new file mode 100644 index 0000000000000..2b2249f8a5831 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/ShareSessionNotFoundException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +/** + * Thrown when the share session was not found. + */ +public class ShareSessionNotFoundException extends RetriableException { + private static final long serialVersionUID = 1L; + + public ShareSessionNotFoundException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java b/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java index 573e2c47a082c..8c31e64cc5140 100644 --- a/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java @@ -26,7 +26,7 @@ /** * Represents an immutable basic version range using 2 attributes: min and max, each of type short. * The min and max attributes need to satisfy 2 rules: - * - they are each expected to be >= 0, as we only consider positive version values to be valid. + * - they are each expected to be >= 0, as we only consider non-negative version values to be valid. * - max should be >= min. * * The class also provides API to convert the version range to a map. diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java index 7367e966c014e..a77cc9309bd20 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java @@ -25,6 +25,8 @@ */ public class MetricConfig { + public static final int DEFAULT_NUM_SAMPLES = 2; + private Quota quota; private int samples; private long eventWindow; @@ -34,7 +36,7 @@ public class MetricConfig { public MetricConfig() { this.quota = null; - this.samples = 2; + this.samples = DEFAULT_NUM_SAMPLES; this.eventWindow = Long.MAX_VALUE; this.timeWindowMs = TimeUnit.MILLISECONDS.convert(30, TimeUnit.SECONDS); this.tags = new LinkedHashMap<>(); 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 09b7c05c8f283..4f15bb9607e1e 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 @@ -74,15 +74,18 @@ public long windowSize(MetricConfig config, long now) { /* * Here we check the total amount of time elapsed since the oldest non-obsolete window. * This give the total windowSize of the batch which is the time used for Rate computation. - * However, there is an issue if we do not have sufficient data for e.g. if only 1 second has elapsed in a 30 second + * However, there is an issue if we do not have sufficient data for e.g. if only 1 second has elapsed in a 30-second * window, the measured rate will be very high. - * Hence we assume that the elapsed time is always N-1 complete windows plus whatever fraction of the final window is complete. + * Hence, we assume that the elapsed time is always N-1 complete windows plus whatever fraction of the final window is complete. * * Note that we could simply count the amount of time elapsed in the current window and add n-1 windows to get the total time, * but this approach does not account for sleeps. SampledStat only creates samples whenever record is called, * if no record is called for a period of time that time is not accounted for in windowSize and produces incorrect results. + * + * Note also, that totalElapsedTimeMs can be larger than the monitored window size, + * if the oldest sample started before the window while overlapping it. */ - long totalElapsedTimeMs = now - stat.oldest(now).lastWindowMs; + long totalElapsedTimeMs = now - stat.oldest(now).startTimeMs; // Check how many full windows of data we have currently retained int numFullWindows = (int) (totalElapsedTimeMs / config.timeWindowMs()); int minFullWindows = config.samples() - 1; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java index df4a95ccaa9b6..f76fccc853bfa 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java @@ -40,7 +40,8 @@ public abstract class SampledStat implements MeasurableStat { public SampledStat(double initialValue) { this.initialValue = initialValue; - this.samples = new ArrayList<>(2); + // keep one extra placeholder for "overlapping sample" (see purgeObsoleteSamples() logic) + this.samples = new ArrayList<>(MetricConfig.DEFAULT_NUM_SAMPLES + 1); } @Override @@ -50,10 +51,13 @@ public void record(MetricConfig config, double value, long timeMs) { sample = advance(config, timeMs); update(sample, config, value, timeMs); sample.eventCount += 1; + sample.lastEventMs = timeMs; } private Sample advance(MetricConfig config, long timeMs) { - this.current = (this.current + 1) % config.samples(); + // keep one extra placeholder for "overlapping sample" (see purgeObsoleteSamples() logic) + int maxSamples = config.samples() + 1; + this.current = (this.current + 1) % maxSamples; if (this.current >= samples.size()) { Sample sample = newSample(timeMs); this.samples.add(sample); @@ -87,7 +91,7 @@ public Sample oldest(long now) { Sample oldest = this.samples.get(0); for (int i = 1; i < this.samples.size(); i++) { Sample curr = this.samples.get(i); - if (curr.lastWindowMs < oldest.lastWindowMs) + if (curr.startTimeMs < oldest.startTimeMs) oldest = curr; } return oldest; @@ -106,36 +110,42 @@ public String toString() { public abstract double combine(List samples, MetricConfig config, long now); - /* Timeout any windows that have expired in the absence of any events */ + // purge any samples that lack observed events within the monitored window protected void purgeObsoleteSamples(MetricConfig config, long now) { long expireAge = config.samples() * config.timeWindowMs(); for (Sample sample : samples) { - if (now - sample.lastWindowMs >= expireAge) + // samples overlapping the monitored window are kept, + // even if they started before it + if (now - sample.lastEventMs >= expireAge) { sample.reset(now); + } } } protected static class Sample { public double initialValue; public long eventCount; - public long lastWindowMs; + public long startTimeMs; + public long lastEventMs; public double value; public Sample(double initialValue, long now) { this.initialValue = initialValue; this.eventCount = 0; - this.lastWindowMs = now; + this.startTimeMs = now; + this.lastEventMs = now; this.value = initialValue; } public void reset(long now) { this.eventCount = 0; - this.lastWindowMs = now; + this.startTimeMs = now; + this.lastEventMs = now; this.value = initialValue; } public boolean isComplete(long timeMs, MetricConfig config) { - return timeMs - lastWindowMs >= config.timeWindowMs() || eventCount >= config.eventWindow(); + return timeMs - startTimeMs >= config.timeWindowMs() || eventCount >= config.eventWindow(); } @Override @@ -143,7 +153,8 @@ public String toString() { return "Sample(" + "value=" + value + ", eventCount=" + eventCount + - ", lastWindowMs=" + lastWindowMs + + ", startTimeMs=" + startTimeMs + + ", lastEventMs=" + lastEventMs + ", initialValue=" + initialValue + ')'; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SimpleRate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SimpleRate.java index 931bd9c35e51f..a632f0254d6b0 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/SimpleRate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SimpleRate.java @@ -33,7 +33,7 @@ public class SimpleRate extends Rate { @Override public long windowSize(MetricConfig config, long now) { stat.purgeObsoleteSamples(config, now); - long elapsed = now - stat.oldest(now).lastWindowMs; + long elapsed = now - stat.oldest(now).startTimeMs; return Math.max(elapsed, config.timeWindowMs()); } } diff --git a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java index b1257c33c4615..1f91fb8d90ec0 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -315,7 +315,7 @@ private void createServerCallbackHandlers(Map configs) { String prefix = ListenerName.saslMechanismPrefix(mechanism); @SuppressWarnings("unchecked") Class clazz = - (Class) configs.get(prefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS); + (Class) configs.get(prefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG); if (clazz != null) callbackHandler = Utils.newInstance(clazz); else if (mechanism.equals(PlainSaslServer.PLAIN_MECHANISM)) @@ -333,9 +333,9 @@ else if (mechanism.equals(OAuthBearerLoginModule.OAUTHBEARER_MECHANISM)) private void createConnectionsMaxReauthMsMap(Map configs) { for (String mechanism : jaasContexts.keySet()) { String prefix = ListenerName.saslMechanismPrefix(mechanism); - Long connectionsMaxReauthMs = (Long) configs.get(prefix + BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS); + Long connectionsMaxReauthMs = (Long) configs.get(prefix + BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG); if (connectionsMaxReauthMs == null) - connectionsMaxReauthMs = (Long) configs.get(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS); + connectionsMaxReauthMs = (Long) configs.get(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG); if (connectionsMaxReauthMs != null) connectionsMaxReauthMsByMechanism.put(mechanism, connectionsMaxReauthMs); } 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 16bec4fb72dc6..ffd5737ca3162 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 @@ -118,7 +118,11 @@ public enum ApiKeys { PUSH_TELEMETRY(ApiMessageType.PUSH_TELEMETRY), ASSIGN_REPLICAS_TO_DIRS(ApiMessageType.ASSIGN_REPLICAS_TO_DIRS), LIST_CLIENT_METRICS_RESOURCES(ApiMessageType.LIST_CLIENT_METRICS_RESOURCES), - DESCRIBE_TOPIC_PARTITIONS(ApiMessageType.DESCRIBE_TOPIC_PARTITIONS); + DESCRIBE_TOPIC_PARTITIONS(ApiMessageType.DESCRIBE_TOPIC_PARTITIONS), + SHARE_GROUP_HEARTBEAT(ApiMessageType.SHARE_GROUP_HEARTBEAT), + SHARE_GROUP_DESCRIBE(ApiMessageType.SHARE_GROUP_DESCRIBE), + SHARE_FETCH(ApiMessageType.SHARE_FETCH), + SHARE_ACKNOWLEDGE(ApiMessageType.SHARE_ACKNOWLEDGE); private static final Map> APIS_BY_LISTENER = new EnumMap<>(ApiMessageType.ListenerType.class); 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 900d191c8f9d4..10ae05aa850c9 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 @@ -41,6 +41,7 @@ import org.apache.kafka.common.errors.FencedInstanceIdException; import org.apache.kafka.common.errors.FencedLeaderEpochException; import org.apache.kafka.common.errors.FencedMemberEpochException; +import org.apache.kafka.common.errors.FencedStateEpochException; import org.apache.kafka.common.errors.FetchSessionIdNotFoundException; import org.apache.kafka.common.errors.FetchSessionTopicIdException; import org.apache.kafka.common.errors.GroupAuthorizationException; @@ -64,12 +65,14 @@ import org.apache.kafka.common.errors.InvalidPidMappingException; import org.apache.kafka.common.errors.InvalidPrincipalTypeException; import org.apache.kafka.common.errors.InvalidProducerEpochException; +import org.apache.kafka.common.errors.InvalidRecordStateException; import org.apache.kafka.common.errors.InvalidRegistrationException; import org.apache.kafka.common.errors.InvalidReplicaAssignmentException; import org.apache.kafka.common.errors.InvalidReplicationFactorException; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.InvalidRequiredAcksException; import org.apache.kafka.common.errors.InvalidSessionTimeoutException; +import org.apache.kafka.common.errors.InvalidShareSessionEpochException; import org.apache.kafka.common.errors.InvalidTimestampException; import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.InvalidTxnStateException; @@ -109,6 +112,7 @@ import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.SaslAuthenticationException; import org.apache.kafka.common.errors.SecurityDisabledException; +import org.apache.kafka.common.errors.ShareSessionNotFoundException; import org.apache.kafka.common.errors.SnapshotNotFoundException; import org.apache.kafka.common.errors.StaleBrokerEpochException; import org.apache.kafka.common.errors.StaleMemberEpochException; @@ -394,7 +398,11 @@ public enum Errors { UNKNOWN_SUBSCRIPTION_ID(117, "Client sent a push telemetry request with an invalid or outdated subscription ID.", UnknownSubscriptionIdException::new), TELEMETRY_TOO_LARGE(118, "Client sent a push telemetry request larger than the maximum size the broker will accept.", TelemetryTooLargeException::new), INVALID_REGISTRATION(119, "The controller has considered the broker registration to be invalid.", InvalidRegistrationException::new), - TRANSACTION_ABORTABLE(120, "The server encountered an error with the transaction. The client can abort the transaction to continue using this transactional ID.", TransactionAbortableException::new); + TRANSACTION_ABORTABLE(120, "The server encountered an error with the transaction. The client can abort the transaction to continue using this transactional ID.", TransactionAbortableException::new), + INVALID_RECORD_STATE(121, "The record state is invalid. The acknowledgement of delivery could not be completed.", InvalidRecordStateException::new), + SHARE_SESSION_NOT_FOUND(122, "The share session was not found.", ShareSessionNotFoundException::new), + INVALID_SHARE_SESSION_EPOCH(123, "The share session epoch is invalid.", InvalidShareSessionEpochException::new), + FENCED_STATE_EPOCH(124, "The share coordinator rejected the request because the share-group state epoch did not match.", FencedStateEpochException::new); private static final Logger log = LoggerFactory.getLogger(Errors.class); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index b51221f5af642..589e163992b22 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -326,6 +326,14 @@ private static AbstractRequest doParseRequest(ApiKeys apiKey, short apiVersion, return ListClientMetricsResourcesRequest.parse(buffer, apiVersion); case DESCRIBE_TOPIC_PARTITIONS: return DescribeTopicPartitionsRequest.parse(buffer, apiVersion); + case SHARE_GROUP_HEARTBEAT: + return ShareGroupHeartbeatRequest.parse(buffer, apiVersion); + case SHARE_GROUP_DESCRIBE: + return ShareGroupDescribeRequest.parse(buffer, apiVersion); + case SHARE_FETCH: + return ShareFetchRequest.parse(buffer, apiVersion); + case SHARE_ACKNOWLEDGE: + return ShareAcknowledgeRequest.parse(buffer, apiVersion); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " + "code should be updated to do so.", apiKey)); 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 dbafdbf3bcb07..5534168098e9d 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 @@ -263,6 +263,14 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, ByteBuffer response return ListClientMetricsResourcesResponse.parse(responseBuffer, version); case DESCRIBE_TOPIC_PARTITIONS: return DescribeTopicPartitionsResponse.parse(responseBuffer, version); + case SHARE_GROUP_HEARTBEAT: + return ShareGroupHeartbeatResponse.parse(responseBuffer, version); + case SHARE_GROUP_DESCRIBE: + return ShareGroupDescribeResponse.parse(responseBuffer, version); + case SHARE_FETCH: + return ShareFetchResponse.parse(responseBuffer, version); + case SHARE_ACKNOWLEDGE: + return ShareAcknowledgeResponse.parse(responseBuffer, version); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java new file mode 100644 index 0000000000000..1b77b43be33c1 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.requests; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.ShareAcknowledgeRequestData; +import org.apache.kafka.common.message.ShareAcknowledgeResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.Errors; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ShareAcknowledgeRequest extends AbstractRequest { + + public static class Builder extends AbstractRequest.Builder { + + private final ShareAcknowledgeRequestData data; + + public Builder(ShareAcknowledgeRequestData data) { + this(data, false); + } + + public Builder(ShareAcknowledgeRequestData data, boolean enableUnstableLastVersion) { + super(ApiKeys.SHARE_ACKNOWLEDGE, enableUnstableLastVersion); + this.data = data; + } + + public static ShareAcknowledgeRequest.Builder forConsumer(String groupId, ShareFetchMetadata metadata, + Map> acknowledgementsMap) { + ShareAcknowledgeRequestData data = new ShareAcknowledgeRequestData(); + data.setGroupId(groupId); + if (metadata != null) { + data.setMemberId(metadata.memberId().toString()); + data.setShareSessionEpoch(metadata.epoch()); + } + + // Build a map of topics to acknowledge keyed by topic ID, and within each a map of partitions keyed by index + Map> ackMap = new HashMap<>(); + + for (Map.Entry> acknowledgeEntry : acknowledgementsMap.entrySet()) { + TopicIdPartition tip = acknowledgeEntry.getKey(); + Map partMap = ackMap.computeIfAbsent(tip.topicId(), k -> new HashMap<>()); + ShareAcknowledgeRequestData.AcknowledgePartition ackPartition = partMap.get(tip.partition()); + if (ackPartition == null) { + ackPartition = new ShareAcknowledgeRequestData.AcknowledgePartition() + .setPartitionIndex(tip.partition()); + partMap.put(tip.partition(), ackPartition); + } + ackPartition.setAcknowledgementBatches(acknowledgeEntry.getValue()); + } + + // Finally, build up the data to fetch + data.setTopics(new ArrayList<>()); + ackMap.forEach((topicId, partMap) -> { + ShareAcknowledgeRequestData.AcknowledgeTopic ackTopic = new ShareAcknowledgeRequestData.AcknowledgeTopic() + .setTopicId(topicId) + .setPartitions(new ArrayList<>()); + data.topics().add(ackTopic); + + partMap.forEach((index, ackPartition) -> ackTopic.partitions().add(ackPartition)); + }); + + return new ShareAcknowledgeRequest.Builder(data, true); + } + + public ShareAcknowledgeRequestData data() { + return data; + } + + @Override + public ShareAcknowledgeRequest build(short version) { + return new ShareAcknowledgeRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + private final ShareAcknowledgeRequestData data; + + public ShareAcknowledgeRequest(ShareAcknowledgeRequestData data, short version) { + super(ApiKeys.SHARE_ACKNOWLEDGE, version); + this.data = data; + } + + @Override + public ShareAcknowledgeRequestData data() { + return data; + } + + @Override + public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { + Errors error = Errors.forException(e); + return new ShareAcknowledgeResponse(new ShareAcknowledgeResponseData() + .setThrottleTimeMs(throttleTimeMs) + .setErrorCode(error.code())); + } + + public static ShareAcknowledgeRequest parse(ByteBuffer buffer, short version) { + return new ShareAcknowledgeRequest( + new ShareAcknowledgeRequestData(new ByteBufferAccessor(buffer), version), + version + ); + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java new file mode 100644 index 0000000000000..5cab233dccac8 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.requests; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.ShareAcknowledgeResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.Errors; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +/** + * Possible error codes. + * - {@link Errors#GROUP_AUTHORIZATION_FAILED} + * - {@link Errors#TOPIC_AUTHORIZATION_FAILED} + * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} + * - {@link Errors#NOT_LEADER_OR_FOLLOWER} + * - {@link Errors#UNKNOWN_TOPIC_ID} + * - {@link Errors#INVALID_RECORD_STATE} + * - {@link Errors#KAFKA_STORAGE_ERROR} + * - {@link Errors#INVALID_REQUEST} + * - {@link Errors#UNKNOWN_SERVER_ERROR} + */ +public class ShareAcknowledgeResponse extends AbstractResponse { + + private final ShareAcknowledgeResponseData data; + + public ShareAcknowledgeResponse(ShareAcknowledgeResponseData data) { + super(ApiKeys.SHARE_ACKNOWLEDGE); + this.data = data; + } + + public Errors error() { + return Errors.forCode(data.errorCode()); + } + + @Override + public ShareAcknowledgeResponseData data() { + return data; + } + + @Override + public Map errorCounts() { + HashMap counts = new HashMap<>(); + updateErrorCounts(counts, Errors.forCode(data.errorCode())); + data.responses().forEach( + topic -> topic.partitions().forEach( + partition -> updateErrorCounts(counts, Errors.forCode(partition.errorCode())) + ) + ); + return counts; + } + + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + + public static ShareAcknowledgeResponse parse(ByteBuffer buffer, short version) { + return new ShareAcknowledgeResponse( + new ShareAcknowledgeResponseData(new ByteBufferAccessor(buffer), version) + ); + } + + private static boolean matchingTopic(ShareAcknowledgeResponseData.ShareAcknowledgeTopicResponse previousTopic, TopicIdPartition currentTopic) { + if (previousTopic == null) + return false; + return previousTopic.topicId().equals(currentTopic.topicId()); + } + + public static ShareAcknowledgeResponseData.PartitionData partitionResponse(TopicIdPartition topicIdPartition, Errors error) { + return partitionResponse(topicIdPartition.topicPartition().partition(), error); + } + + public static ShareAcknowledgeResponseData.PartitionData partitionResponse(int partition, Errors error) { + return new ShareAcknowledgeResponseData.PartitionData() + .setPartitionIndex(partition) + .setErrorCode(error.code()); + } + + public static ShareAcknowledgeResponse of(Errors error, + int throttleTimeMs, + LinkedHashMap responseData, + List nodeEndpoints) { + return new ShareAcknowledgeResponse(toMessage(error, throttleTimeMs, responseData.entrySet().iterator(), nodeEndpoints)); + } + + public static ShareAcknowledgeResponseData toMessage(Errors error, int throttleTimeMs, + Iterator> partIterator, + List nodeEndpoints) { + Map topicResponseList = new LinkedHashMap<>(); + while (partIterator.hasNext()) { + Map.Entry entry = partIterator.next(); + ShareAcknowledgeResponseData.PartitionData partitionData = entry.getValue(); + // Since PartitionData alone doesn't know the partition ID, we set it here + partitionData.setPartitionIndex(entry.getKey().topicPartition().partition()); + // Checking if the topic is already present in the map + if (topicResponseList.containsKey(entry.getKey().topicId())) { + topicResponseList.get(entry.getKey().topicId()).partitions().add(partitionData); + } else { + List partitionResponses = new ArrayList<>(); + partitionResponses.add(partitionData); + topicResponseList.put(entry.getKey().topicId(), new ShareAcknowledgeResponseData.ShareAcknowledgeTopicResponse() + .setTopicId(entry.getKey().topicId()) + .setPartitions(partitionResponses)); + } + } + ShareAcknowledgeResponseData data = new ShareAcknowledgeResponseData(); + // KafkaApis should only pass in node endpoints on error, otherwise this should be an empty list + nodeEndpoints.forEach(endpoint -> data.nodeEndpoints().add( + new ShareAcknowledgeResponseData.NodeEndpoint() + .setNodeId(endpoint.id()) + .setHost(endpoint.host()) + .setPort(endpoint.port()) + .setRack(endpoint.rack()))); + return data.setThrottleTimeMs(throttleTimeMs) + .setErrorCode(error.code()) + .setResponses(new ArrayList<>(topicResponseList.values())); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchMetadata.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchMetadata.java new file mode 100644 index 0000000000000..4e5bcc2237e43 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchMetadata.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.common.requests; + +import org.apache.kafka.common.Uuid; + +public class ShareFetchMetadata { + /** + * The first epoch. When used in a ShareFetch request, indicates that the client + * wants to create a session. + */ + public static final int INITIAL_EPOCH = 0; + + /** + * An invalid epoch. When used in a ShareFetch request, indicates that the client + * wants to close an existing session. + */ + public static final int FINAL_EPOCH = -1; + + /** + * + */ + public boolean isNewSession() { + return epoch == INITIAL_EPOCH; + } + + /** + * Returns true if this is a full share fetch request. + */ + public boolean isFull() { + return (this.epoch == INITIAL_EPOCH) || (this.epoch == FINAL_EPOCH); + } + + /** + * Returns the next epoch. + * + * @param prevEpoch The previous epoch. + * @return The next epoch. + */ + public static int nextEpoch(int prevEpoch) { + if (prevEpoch < 0) { + // The next epoch after FINAL_EPOCH is always FINAL_EPOCH itself. + return FINAL_EPOCH; + } else if (prevEpoch == Integer.MAX_VALUE) { + return 1; + } else { + return prevEpoch + 1; + } + } + + /** + * The member ID. + */ + private final Uuid memberId; + + /** + * The share session epoch. + */ + private final int epoch; + + public ShareFetchMetadata(Uuid memberId, int epoch) { + this.memberId = memberId; + this.epoch = epoch; + } + + public static ShareFetchMetadata initialEpoch(Uuid memberId) { + return new ShareFetchMetadata(memberId, INITIAL_EPOCH); + } + + public ShareFetchMetadata nextEpoch() { + return new ShareFetchMetadata(memberId, nextEpoch(epoch)); + } + + public ShareFetchMetadata nextCloseExistingAttemptNew() { + return new ShareFetchMetadata(memberId, INITIAL_EPOCH); + } + + public ShareFetchMetadata finalEpoch() { + return new ShareFetchMetadata(memberId, FINAL_EPOCH); + } + + public Uuid memberId() { + return memberId; + } + + public int epoch() { + return epoch; + } + + public boolean isFinalEpoch() { + return epoch == FINAL_EPOCH; + } + + public String toString() { + StringBuilder bld = new StringBuilder(); + bld.append("(memberId=").append(memberId).append(", "); + if (epoch == INITIAL_EPOCH) { + bld.append("epoch=INITIAL)"); + } else if (epoch == FINAL_EPOCH) { + bld.append("epoch=FINAL)"); + } else { + bld.append("epoch=").append(epoch).append(")"); + } + return bld.toString(); + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java new file mode 100644 index 0000000000000..385e802a691a9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.requests; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.ShareFetchRequestData; +import org.apache.kafka.common.message.ShareFetchResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.Errors; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class ShareFetchRequest extends AbstractRequest { + + public static class Builder extends AbstractRequest.Builder { + + private final ShareFetchRequestData data; + + public Builder(ShareFetchRequestData data) { + this(data, false); + } + + public Builder(ShareFetchRequestData data, boolean enableUnstableLastVersion) { + super(ApiKeys.SHARE_FETCH, enableUnstableLastVersion); + this.data = data; + } + + public static Builder forConsumer(String groupId, ShareFetchMetadata metadata, + int maxWait, int minBytes, int maxBytes, int fetchSize, + List send, List forget, + Map> acknowledgementsMap) { + ShareFetchRequestData data = new ShareFetchRequestData(); + data.setGroupId(groupId); + int ackOnlyPartitionMaxBytes = fetchSize; + boolean isClosingShareSession = false; + if (metadata != null) { + data.setMemberId(metadata.memberId().toString()); + data.setShareSessionEpoch(metadata.epoch()); + if (metadata.isFinalEpoch()) { + isClosingShareSession = true; + ackOnlyPartitionMaxBytes = 0; + } + } + data.setMaxWaitMs(maxWait); + data.setMinBytes(minBytes); + data.setMaxBytes(maxBytes); + + // Build a map of topics to fetch keyed by topic ID, and within each a map of partitions keyed by index + Map> fetchMap = new HashMap<>(); + + // First, start by adding the list of topic-partitions we are fetching + if (!isClosingShareSession) { + for (TopicIdPartition tip : send) { + Map partMap = fetchMap.computeIfAbsent(tip.topicId(), k -> new HashMap<>()); + ShareFetchRequestData.FetchPartition fetchPartition = new ShareFetchRequestData.FetchPartition() + .setPartitionIndex(tip.partition()) + .setPartitionMaxBytes(fetchSize); + partMap.put(tip.partition(), fetchPartition); + } + } + + // Next, add acknowledgements that we are piggybacking onto the fetch. Generally, the list of + // topic-partitions will be a subset, but if the assignment changes, there might be new entries to add + for (Map.Entry> acknowledgeEntry : acknowledgementsMap.entrySet()) { + TopicIdPartition tip = acknowledgeEntry.getKey(); + Map partMap = fetchMap.computeIfAbsent(tip.topicId(), k -> new HashMap<>()); + ShareFetchRequestData.FetchPartition fetchPartition = partMap.get(tip.partition()); + if (fetchPartition == null) { + fetchPartition = new ShareFetchRequestData.FetchPartition() + .setPartitionIndex(tip.partition()) + .setPartitionMaxBytes(ackOnlyPartitionMaxBytes); + partMap.put(tip.partition(), fetchPartition); + } + fetchPartition.setAcknowledgementBatches(acknowledgeEntry.getValue()); + } + + // Build up the data to fetch + if (!fetchMap.isEmpty()) { + data.setTopics(new ArrayList<>()); + fetchMap.forEach((topicId, partMap) -> { + ShareFetchRequestData.FetchTopic fetchTopic = new ShareFetchRequestData.FetchTopic() + .setTopicId(topicId) + .setPartitions(new ArrayList<>()); + partMap.forEach((index, fetchPartition) -> fetchTopic.partitions().add(fetchPartition)); + data.topics().add(fetchTopic); + }); + } + + // And finally, forget the topic-partitions that are no longer in the session + if (!forget.isEmpty()) { + Map> forgetMap = new HashMap<>(); + for (TopicIdPartition tip : forget) { + List partList = forgetMap.computeIfAbsent(tip.topicId(), k -> new ArrayList<>()); + partList.add(tip.partition()); + } + data.setForgottenTopicsData(new ArrayList<>()); + forgetMap.forEach((topicId, partList) -> { + ShareFetchRequestData.ForgottenTopic forgetTopic = new ShareFetchRequestData.ForgottenTopic() + .setTopicId(topicId) + .setPartitions(new ArrayList<>()); + partList.forEach(index -> forgetTopic.partitions().add(index)); + data.forgottenTopicsData().add(forgetTopic); + }); + } + + return new Builder(data, true); + } + + public ShareFetchRequestData data() { + return data; + } + + @Override + public ShareFetchRequest build(short version) { + return new ShareFetchRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + private final ShareFetchRequestData data; + private volatile LinkedHashMap shareFetchData = null; + private volatile List toForget = null; + + public ShareFetchRequest(ShareFetchRequestData data, short version) { + super(ApiKeys.SHARE_FETCH, version); + this.data = data; + } + + @Override + public ShareFetchRequestData data() { + return data; + } + + @Override + public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { + Errors error = Errors.forException(e); + return new ShareFetchResponse(new ShareFetchResponseData() + .setThrottleTimeMs(throttleTimeMs) + .setErrorCode(error.code())); + } + + public static ShareFetchRequest parse(ByteBuffer buffer, short version) { + return new ShareFetchRequest( + new ShareFetchRequestData(new ByteBufferAccessor(buffer), version), + version + ); + } + + public static final class SharePartitionData { + public final Uuid topicId; + public final int maxBytes; + + public SharePartitionData( + Uuid topicId, + int maxBytes + ) { + this.topicId = topicId; + this.maxBytes = maxBytes; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ShareFetchRequest.SharePartitionData that = (ShareFetchRequest.SharePartitionData) o; + return Objects.equals(topicId, that.topicId) && + maxBytes == that.maxBytes; + } + + @Override + public int hashCode() { + return Objects.hash(topicId, maxBytes); + } + + @Override + public String toString() { + return "SharePartitionData(" + + "topicId=" + topicId + + ", maxBytes=" + maxBytes + + ')'; + } + } + + public int minBytes() { + return data.minBytes(); + } + + public int maxBytes() { + return data.maxBytes(); + } + + public int maxWait() { + return data.maxWaitMs(); + } + + public Map shareFetchData(Map topicNames) { + if (shareFetchData == null) { + synchronized (this) { + if (shareFetchData == null) { + // Assigning the lazy-initialized `shareFetchData` in the last step + // to avoid other threads accessing a half-initialized object. + final LinkedHashMap shareFetchDataTmp = new LinkedHashMap<>(); + data.topics().forEach(shareFetchTopic -> { + String name = topicNames.get(shareFetchTopic.topicId()); + shareFetchTopic.partitions().forEach(shareFetchPartition -> { + // Topic name may be null here if the topic name was unable to be resolved using the topicNames map. + shareFetchDataTmp.put(new TopicIdPartition(shareFetchTopic.topicId(), new TopicPartition(name, shareFetchPartition.partitionIndex())), + new ShareFetchRequest.SharePartitionData( + shareFetchTopic.topicId(), + shareFetchPartition.partitionMaxBytes() + ) + ); + }); + }); + shareFetchData = shareFetchDataTmp; + } + } + } + return shareFetchData; + } + + public List forgottenTopics(Map topicNames) { + if (toForget == null) { + synchronized (this) { + if (toForget == null) { + // Assigning the lazy-initialized `toForget` in the last step + // to avoid other threads accessing a half-initialized object. + final List toForgetTmp = new ArrayList<>(); + data.forgottenTopicsData().forEach(forgottenTopic -> { + String name = topicNames.get(forgottenTopic.topicId()); + // Topic name may be null here if the topic name was unable to be resolved using the topicNames map. + forgottenTopic.partitions().forEach(partitionId -> toForgetTmp.add(new TopicIdPartition(forgottenTopic.topicId(), new TopicPartition(name, partitionId)))); + }); + toForget = toForgetTmp; + } + } + } + return toForget; + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java new file mode 100644 index 0000000000000..b33969e0efa41 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.requests; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.ShareFetchResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +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.record.MemoryRecords; +import org.apache.kafka.common.record.Records; + +import java.nio.ByteBuffer; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Iterator; +import java.util.Collections; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + + +/** + * Possible error codes. + * - {@link Errors#GROUP_AUTHORIZATION_FAILED} + * - {@link Errors#TOPIC_AUTHORIZATION_FAILED} + * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} + * - {@link Errors#NOT_LEADER_OR_FOLLOWER} + * - {@link Errors#UNKNOWN_TOPIC_ID} + * - {@link Errors#INVALID_RECORD_STATE} + * - {@link Errors#KAFKA_STORAGE_ERROR} + * - {@link Errors#CORRUPT_MESSAGE} + * - {@link Errors#INVALID_REQUEST} + * - {@link Errors#UNKNOWN_SERVER_ERROR} + */ +public class ShareFetchResponse extends AbstractResponse { + + private final ShareFetchResponseData data; + + private volatile LinkedHashMap responseData = null; + + public ShareFetchResponse(ShareFetchResponseData data) { + super(ApiKeys.SHARE_FETCH); + this.data = data; + } + + public Errors error() { + return Errors.forCode(data.errorCode()); + } + + @Override + public ShareFetchResponseData data() { + return data; + } + + @Override + public Map errorCounts() { + HashMap counts = new HashMap<>(); + updateErrorCounts(counts, Errors.forCode(data.errorCode())); + data.responses().forEach( + topic -> topic.partitions().forEach( + partition -> updateErrorCounts(counts, Errors.forCode(partition.errorCode())) + ) + ); + return counts; + } + + public LinkedHashMap responseData(Map topicNames) { + if (responseData == null) { + synchronized (this) { + // Assigning the lazy-initialized `responseData` in the last step + // to avoid other threads accessing a half-initialized object. + if (responseData == null) { + final LinkedHashMap responseDataTmp = new LinkedHashMap<>(); + data.responses().forEach(topicResponse -> { + String name = topicNames.get(topicResponse.topicId()); + if (name != null) { + topicResponse.partitions().forEach(partitionData -> responseDataTmp.put(new TopicIdPartition(topicResponse.topicId(), + new TopicPartition(name, partitionData.partitionIndex())), partitionData)); + } + }); + responseData = responseDataTmp; + } + } + } + return responseData; + } + + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + + public static ShareFetchResponse parse(ByteBuffer buffer, short version) { + return new ShareFetchResponse( + new ShareFetchResponseData(new ByteBufferAccessor(buffer), version) + ); + } + + /** + * Returns `partition.records` as `Records` (instead of `BaseRecords`). If `records` is `null`, returns `MemoryRecords.EMPTY`. + * + *

If this response was deserialized after a share fetch, this method should never fail. An example where this would + * fail is a down-converted response (e.g. LazyDownConversionRecords) on the broker (before it's serialized and + * sent on the wire). + * + * @param partition partition data + * @return Records or empty record if the records in PartitionData is null. + */ + public static Records recordsOrFail(ShareFetchResponseData.PartitionData partition) { + if (partition.records() == null) return MemoryRecords.EMPTY; + if (partition.records() instanceof Records) return (Records) partition.records(); + throw new ClassCastException("The record type is " + partition.records().getClass().getSimpleName() + ", which is not a subtype of " + + Records.class.getSimpleName() + ". This method is only safe to call if the `ShareFetchResponse` was deserialized from bytes."); + } + + /** + * Convenience method to find the size of a response. + * + * @param version The version of the request + * @param partIterator The partition iterator. + * @return The response size in bytes. + */ + public static int sizeOf(short version, + Iterator> partIterator) { + // Since the throttleTimeMs and metadata field sizes are constant and fixed, we can + // use arbitrary values here without affecting the result. + ShareFetchResponseData data = toMessage(Errors.NONE, 0, partIterator, Collections.emptyList()); + ObjectSerializationCache cache = new ObjectSerializationCache(); + return 4 + data.size(cache, version); + } + + /** + * @return The size in bytes of the records. 0 is returned if records of input partition is null. + */ + public static int recordsSize(ShareFetchResponseData.PartitionData partition) { + return partition.records() == null ? 0 : partition.records().sizeInBytes(); + } + + public static ShareFetchResponse of(Errors error, + int throttleTimeMs, + LinkedHashMap responseData, + List nodeEndpoints) { + return new ShareFetchResponse(toMessage(error, throttleTimeMs, responseData.entrySet().iterator(), nodeEndpoints)); + } + + public static ShareFetchResponseData toMessage(Errors error, int throttleTimeMs, + Iterator> partIterator, + List nodeEndpoints) { + Map topicResponseList = new LinkedHashMap<>(); + while (partIterator.hasNext()) { + Map.Entry entry = partIterator.next(); + ShareFetchResponseData.PartitionData partitionData = entry.getValue(); + // Since PartitionData alone doesn't know the partition ID, we set it here + partitionData.setPartitionIndex(entry.getKey().topicPartition().partition()); + // Checking if the topic is already present in the map + if (topicResponseList.containsKey(entry.getKey().topicId())) { + topicResponseList.get(entry.getKey().topicId()).partitions().add(partitionData); + } else { + List partitionResponses = new ArrayList<>(); + partitionResponses.add(partitionData); + topicResponseList.put(entry.getKey().topicId(), new ShareFetchResponseData.ShareFetchableTopicResponse() + .setTopicId(entry.getKey().topicId()) + .setPartitions(partitionResponses)); + } + } + ShareFetchResponseData data = new ShareFetchResponseData(); + // KafkaApis should only pass in node endpoints on error, otherwise this should be an empty list + nodeEndpoints.forEach(endpoint -> data.nodeEndpoints().add( + new ShareFetchResponseData.NodeEndpoint() + .setNodeId(endpoint.id()) + .setHost(endpoint.host()) + .setPort(endpoint.port()) + .setRack(endpoint.rack()))); + return data.setThrottleTimeMs(throttleTimeMs) + .setErrorCode(error.code()) + .setResponses(new ArrayList<>(topicResponseList.values())); + } + + public static ShareFetchResponseData.PartitionData partitionResponse(TopicIdPartition topicIdPartition, Errors error) { + return partitionResponse(topicIdPartition.topicPartition().partition(), error); + } + + public static ShareFetchResponseData.PartitionData partitionResponse(int partition, Errors error) { + return new ShareFetchResponseData.PartitionData() + .setPartitionIndex(partition) + .setErrorCode(error.code()); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java new file mode 100644 index 0000000000000..25c02e4a83c5e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.requests; + +import org.apache.kafka.common.message.ShareGroupDescribeRequestData; +import org.apache.kafka.common.message.ShareGroupDescribeResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.Errors; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.stream.Collectors; + +public class ShareGroupDescribeRequest extends AbstractRequest { + + public static class Builder extends AbstractRequest.Builder { + + private final ShareGroupDescribeRequestData data; + + public Builder(ShareGroupDescribeRequestData data) { + this(data, false); + } + + public Builder(ShareGroupDescribeRequestData data, boolean enableUnstableLastVersion) { + super(ApiKeys.SHARE_GROUP_DESCRIBE, enableUnstableLastVersion); + this.data = data; + } + + @Override + public ShareGroupDescribeRequest build(short version) { + return new ShareGroupDescribeRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + private final ShareGroupDescribeRequestData data; + + public ShareGroupDescribeRequest(ShareGroupDescribeRequestData data, short version) { + super(ApiKeys.SHARE_GROUP_DESCRIBE, version); + this.data = data; + } + + @Override + public ShareGroupDescribeResponse getErrorResponse(int throttleTimeMs, Throwable e) { + ShareGroupDescribeResponseData data = new ShareGroupDescribeResponseData() + .setThrottleTimeMs(throttleTimeMs); + // Set error for each group + short errorCode = Errors.forException(e).code(); + this.data.groupIds().forEach( + groupId -> data.groups().add( + new ShareGroupDescribeResponseData.DescribedGroup() + .setGroupId(groupId) + .setErrorCode(errorCode) + ) + ); + return new ShareGroupDescribeResponse(data); + } + + @Override + public ShareGroupDescribeRequestData data() { + return data; + } + + public static ShareGroupDescribeRequest parse(ByteBuffer buffer, short version) { + return new ShareGroupDescribeRequest( + new ShareGroupDescribeRequestData(new ByteBufferAccessor(buffer), version), + version + ); + } + + public static List getErrorDescribedGroupList( + List groupIds, + Errors error + ) { + return groupIds.stream() + .map(groupId -> new ShareGroupDescribeResponseData.DescribedGroup() + .setGroupId(groupId) + .setErrorCode(error.code()) + ).collect(Collectors.toList()); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeResponse.java new file mode 100644 index 0000000000000..95dd371eedfa7 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeResponse.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.requests; + +import org.apache.kafka.common.message.ShareGroupDescribeResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.Errors; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +/** + * Possible error codes. + * + * - {@link Errors#GROUP_AUTHORIZATION_FAILED} + * - {@link Errors#NOT_COORDINATOR} + * - {@link Errors#COORDINATOR_NOT_AVAILABLE} + * - {@link Errors#COORDINATOR_LOAD_IN_PROGRESS} + * - {@link Errors#INVALID_REQUEST} + * - {@link Errors#INVALID_GROUP_ID} + * - {@link Errors#GROUP_ID_NOT_FOUND} + */ +public class ShareGroupDescribeResponse extends AbstractResponse { + + private final ShareGroupDescribeResponseData data; + + public ShareGroupDescribeResponse(ShareGroupDescribeResponseData data) { + super(ApiKeys.SHARE_GROUP_DESCRIBE); + this.data = data; + } + + @Override + public ShareGroupDescribeResponseData data() { + return data; + } + + @Override + public Map errorCounts() { + HashMap counts = new HashMap<>(); + data.groups().forEach( + group -> updateErrorCounts(counts, Errors.forCode(group.errorCode())) + ); + return counts; + } + + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + + public static ShareGroupDescribeResponse parse(ByteBuffer buffer, short version) { + return new ShareGroupDescribeResponse( + new ShareGroupDescribeResponseData(new ByteBufferAccessor(buffer), version) + ); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatRequest.java new file mode 100644 index 0000000000000..7e112ef29dd14 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatRequest.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.requests; + +import java.nio.ByteBuffer; + +import org.apache.kafka.common.message.ShareGroupHeartbeatRequestData; +import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.Errors; + +public class ShareGroupHeartbeatRequest extends AbstractRequest { + /** + * A member epoch of -1 means that the member wants to leave the group. + */ + public static final int LEAVE_GROUP_MEMBER_EPOCH = -1; + + /** + * A member epoch of 0 means that the member wants to join the group. + */ + public static final int JOIN_GROUP_MEMBER_EPOCH = 0; + + public static class Builder extends AbstractRequest.Builder { + private final ShareGroupHeartbeatRequestData data; + + public Builder(ShareGroupHeartbeatRequestData data) { + this(data, true); + } + + public Builder(ShareGroupHeartbeatRequestData data, boolean enableUnstableLastVersion) { + super(ApiKeys.SHARE_GROUP_HEARTBEAT, enableUnstableLastVersion); + this.data = data; + } + + @Override + public ShareGroupHeartbeatRequest build(short version) { + return new ShareGroupHeartbeatRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + private final ShareGroupHeartbeatRequestData data; + + public ShareGroupHeartbeatRequest(ShareGroupHeartbeatRequestData data, short version) { + super(ApiKeys.SHARE_GROUP_HEARTBEAT, version); + this.data = data; + } + + @Override + public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { + return new ShareGroupHeartbeatResponse( + new ShareGroupHeartbeatResponseData() + .setThrottleTimeMs(throttleTimeMs) + .setErrorCode(Errors.forException(e).code()) + ); + } + + @Override + public ShareGroupHeartbeatRequestData data() { + return data; + } + + public static ShareGroupHeartbeatRequest parse(ByteBuffer buffer, short version) { + return new ShareGroupHeartbeatRequest(new ShareGroupHeartbeatRequestData( + new ByteBufferAccessor(buffer), version), version); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatResponse.java new file mode 100644 index 0000000000000..de05d44aebecb --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupHeartbeatResponse.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.requests; + +import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.Errors; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Map; + +/** + * Possible error codes. + * + * - {@link Errors#GROUP_AUTHORIZATION_FAILED} + * - {@link Errors#NOT_COORDINATOR} + * - {@link Errors#COORDINATOR_NOT_AVAILABLE} + * - {@link Errors#COORDINATOR_LOAD_IN_PROGRESS} + * - {@link Errors#INVALID_REQUEST} + * - {@link Errors#UNKNOWN_MEMBER_ID} + * - {@link Errors#GROUP_MAX_SIZE_REACHED} + */ +public class ShareGroupHeartbeatResponse extends AbstractResponse { + private final ShareGroupHeartbeatResponseData data; + + public ShareGroupHeartbeatResponse(ShareGroupHeartbeatResponseData data) { + super(ApiKeys.SHARE_GROUP_HEARTBEAT); + this.data = data; + } + + @Override + public ShareGroupHeartbeatResponseData data() { + return data; + } + + @Override + public Map errorCounts() { + return Collections.singletonMap(Errors.forCode(data.errorCode()), 1); + } + + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + + @Override + public void maybeSetThrottleTimeMs(int throttleTimeMs) { + data.setThrottleTimeMs(throttleTimeMs); + } + + public static ShareGroupHeartbeatResponse parse(ByteBuffer buffer, short version) { + return new ShareGroupHeartbeatResponse(new ShareGroupHeartbeatResponseData( + new ByteBufferAccessor(buffer), version)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslInternalConfigs.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslInternalConfigs.java index c1793ebc3192b..d55ad04636f98 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslInternalConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslInternalConfigs.java @@ -21,7 +21,7 @@ public class SaslInternalConfigs { /** * The server (broker) specifies a positive session length in milliseconds to a - * SASL client when {@link BrokerSecurityConfigs#CONNECTIONS_MAX_REAUTH_MS} is + * SASL client when {@link BrokerSecurityConfigs#CONNECTIONS_MAX_REAUTH_MS_CONFIG} is * positive as per KIP * 368: Allow SASL Connections to Periodically Re-Authenticate. The session 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 06e8dcd0bca5b..ee0ed8007d83c 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 @@ -182,7 +182,7 @@ public SaslServerAuthenticator(Map configs, throw new IllegalArgumentException("Callback handler not specified for SASL mechanism " + mechanism); if (!subjects.containsKey(mechanism)) throw new IllegalArgumentException("Subject cannot be null for SASL mechanism " + mechanism); - LOG.trace("{} for mechanism={}: {}", BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS, mechanism, + LOG.trace("{} for mechanism={}: {}", BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, mechanism, connectionsMaxReauthMsByMechanism.get(mechanism)); } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java index 2d296b545bbca..d39a06787fb54 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerValidatorCallbackHandler.java @@ -55,7 +55,7 @@ * *

* This {@link AuthenticateCallbackHandler} is enabled in the broker configuration by setting the - * {@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS} + * {@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG} * like so: * * @@ -86,7 +86,7 @@ * validation callback handler: * *

    - *
  • {@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS}
  • + *
  • {@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG}
  • *
  • {@link org.apache.kafka.common.config.SaslConfigs#SASL_JAAS_CONFIG}
  • *
  • {@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS}
  • *
  • {@link org.apache.kafka.common.config.SaslConfigs#SASL_OAUTHBEARER_EXPECTED_AUDIENCE}
  • diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index ce67fbdb0c73e..e8e0346a58f2f 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -92,9 +92,9 @@ public final class Utils { private Utils() {} - // This matches URIs of formats: host:port and protocol:\\host:port + // This matches URIs of formats: host:port and protocol://host:port // IPv6 is supported with [ip] pattern - private static final Pattern HOST_PORT_PATTERN = Pattern.compile(".*?\\[?([0-9a-zA-Z\\-%._:]*)\\]?:([0-9]+)"); + private static final Pattern HOST_PORT_PATTERN = Pattern.compile("^(?:[a-zA-Z][a-zA-Z\\d+-.]*://)?\\[?([0-9a-zA-Z\\-._%:]+)\\]?:([0-9]+)$"); private static final Pattern VALID_HOST_CHARACTERS = Pattern.compile("([0-9a-zA-Z\\-%._:]*)"); diff --git a/clients/src/main/resources/common/message/FindCoordinatorRequest.json b/clients/src/main/resources/common/message/FindCoordinatorRequest.json index 42b2f4c891ad5..43e6fe5014b26 100644 --- a/clients/src/main/resources/common/message/FindCoordinatorRequest.json +++ b/clients/src/main/resources/common/message/FindCoordinatorRequest.json @@ -27,7 +27,9 @@ // Version 4 adds support for batching via CoordinatorKeys (KIP-699) // // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). - "validVersions": "0-5", + // + // Version 6 adds support for share groups (KIP-932). + "validVersions": "0-6", "deprecatedVersions": "0", "flexibleVersions": "3+", "fields": [ diff --git a/clients/src/main/resources/common/message/FindCoordinatorResponse.json b/clients/src/main/resources/common/message/FindCoordinatorResponse.json index 860d655a252b2..be0479f908c96 100644 --- a/clients/src/main/resources/common/message/FindCoordinatorResponse.json +++ b/clients/src/main/resources/common/message/FindCoordinatorResponse.json @@ -26,7 +26,9 @@ // Version 4 adds support for batching via Coordinators (KIP-699) // // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). - "validVersions": "0-5", + // + // Version 6 adds support for share groups (KIP-932). + "validVersions": "0-6", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, diff --git a/clients/src/main/resources/common/message/ListGroupsRequest.json b/clients/src/main/resources/common/message/ListGroupsRequest.json index 32defaa203382..a872165d516cf 100644 --- a/clients/src/main/resources/common/message/ListGroupsRequest.json +++ b/clients/src/main/resources/common/message/ListGroupsRequest.json @@ -25,7 +25,9 @@ // Version 4 adds the StatesFilter field (KIP-518). // // Version 5 adds the TypesFilter field (KIP-848). - "validVersions": "0-5", + // + // Version 6 adds support for share groups (KIP-932). + "validVersions": "0-6", "flexibleVersions": "3+", "fields": [ { "name": "StatesFilter", "type": "[]string", "versions": "4+", diff --git a/clients/src/main/resources/common/message/ListGroupsResponse.json b/clients/src/main/resources/common/message/ListGroupsResponse.json index fc4077c080f46..77f1c89e34a38 100644 --- a/clients/src/main/resources/common/message/ListGroupsResponse.json +++ b/clients/src/main/resources/common/message/ListGroupsResponse.json @@ -27,7 +27,9 @@ // Version 4 adds the GroupState field (KIP-518). // // Version 5 adds the GroupType field (KIP-848). - "validVersions": "0-5", + // + // Version 6 adds support for share groups (KIP-932). + "validVersions": "0-6", "flexibleVersions": "3+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, diff --git a/clients/src/main/resources/common/message/ShareAcknowledgeRequest.json b/clients/src/main/resources/common/message/ShareAcknowledgeRequest.json new file mode 100644 index 0000000000000..db534cb4c1c13 --- /dev/null +++ b/clients/src/main/resources/common/message/ShareAcknowledgeRequest.json @@ -0,0 +1,53 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 79, + "type": "request", + "listeners": ["broker"], + "name": "ShareAcknowledgeRequest", + "validVersions": "0", + "flexibleVersions": "0+", + // The ShareAcknowledgeRequest API is added as part of KIP-932 and is still under + // development. Hence, the API is not exposed by default by brokers unless + // explicitly enabled. + "latestVersionUnstable": true, + "fields": [ + { "name": "GroupId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "entityType": "groupId", + "about": "The group identifier." }, + { "name": "MemberId", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The member ID." }, + { "name": "ShareSessionEpoch", "type": "int32", "versions": "0+", + "about": "The current share session epoch: 0 to open a share session; -1 to close it; otherwise increments for consecutive requests." }, + { "name": "Topics", "type": "[]AcknowledgeTopic", "versions": "0+", + "about": "The topics containing records to acknowledge.", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", "about": "The unique topic ID."}, + { "name": "Partitions", "type": "[]AcknowledgePartition", "versions": "0+", + "about": "The partitions containing records to acknowledge.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "AcknowledgementBatches", "type": "[]AcknowledgementBatch", "versions": "0+", + "about": "Record batches to acknowledge.", "fields": [ + { "name": "FirstOffset", "type": "int64", "versions": "0+", + "about": "First offset of batch of records to acknowledge."}, + { "name": "LastOffset", "type": "int64", "versions": "0+", + "about": "Last offset (inclusive) of batch of records to acknowledge."}, + { "name": "AcknowledgeTypes", "type": "[]int8", "versions": "0+", + "about": "Array of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject."} + ]} + ]} + ]} + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareAcknowledgeResponse.json b/clients/src/main/resources/common/message/ShareAcknowledgeResponse.json new file mode 100644 index 0000000000000..638ca10c64b3b --- /dev/null +++ b/clients/src/main/resources/common/message/ShareAcknowledgeResponse.json @@ -0,0 +1,72 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 79, + "type": "response", + "name": "ShareAcknowledgeResponse", + "validVersions": "0", + "flexibleVersions": "0+", + // Supported errors: + // - GROUP_AUTHORIZATION_FAILED (version 0+) + // - TOPIC_AUTHORIZATION_FAILED (version 0+) + // - UNKNOWN_TOPIC_OR_PARTITION (version 0+) + // - SHARE_SESSION_NOT_FOUND (version 0+) + // - INVALID_SHARE_SESSION_EPOCH (version 0+) + // - NOT_LEADER_OR_FOLLOWER (version 0+) + // - UNKNOWN_TOPIC_ID (version 0+) + // - INVALID_RECORD_STATE (version 0+) + // - KAFKA_STORAGE_ERROR (version 0+) + // - INVALID_REQUEST (version 0+) + // - UNKNOWN_SERVER_ERROR (version 0+) + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", "ignorable": true, + "about": "The top level response error code." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The top-level error message, or null if there was no error." }, + { "name": "Responses", "type": "[]ShareAcknowledgeTopicResponse", "versions": "0+", + "about": "The response topics.", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID."}, + { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", + "about": "The topic partitions.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The error message, or null if there was no error." }, + { "name": "CurrentLeader", "type": "LeaderIdAndEpoch", "versions": "0+", "fields": [ + { "name": "LeaderId", "type": "int32", "versions": "0+", + "about": "The ID of the current leader or -1 if the leader is unknown." }, + { "name": "LeaderEpoch", "type": "int32", "versions": "0+", + "about": "The latest known leader epoch." } + ]} + ]} + ]}, + { "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "0+", + "about": "Endpoints for all current leaders enumerated in PartitionData with error NOT_LEADER_OR_FOLLOWER.", "fields": [ + { "name": "NodeId", "type": "int32", "versions": "0+", + "mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node." }, + { "name": "Host", "type": "string", "versions": "0+", + "about": "The node's hostname." }, + { "name": "Port", "type": "int32", "versions": "0+", + "about": "The node's port." }, + { "name": "Rack", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The rack of the node, or null if it has not been assigned to a rack." } + ]} + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareFetchRequest.json b/clients/src/main/resources/common/message/ShareFetchRequest.json new file mode 100644 index 0000000000000..d0b59dcb26a80 --- /dev/null +++ b/clients/src/main/resources/common/message/ShareFetchRequest.json @@ -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. + +{ + "apiKey": 78, + "type": "request", + "listeners": ["broker"], + "name": "ShareFetchRequest", + "validVersions": "0", + "flexibleVersions": "0+", + // The ShareFetchRequest API is added as part of KIP-932 and is still under + // development. Hence, the API is not exposed by default by brokers unless + // explicitly enabled. + "latestVersionUnstable": true, + "fields": [ + { "name": "GroupId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "entityType": "groupId", + "about": "The group identifier." }, + { "name": "MemberId", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The member ID." }, + { "name": "ShareSessionEpoch", "type": "int32", "versions": "0+", + "about": "The current share session epoch: 0 to open a share session; -1 to close it; otherwise increments for consecutive requests." }, + { "name": "MaxWaitMs", "type": "int32", "versions": "0+", + "about": "The maximum time in milliseconds to wait for the response." }, + { "name": "MinBytes", "type": "int32", "versions": "0+", + "about": "The minimum bytes to accumulate in the response." }, + { "name": "MaxBytes", "type": "int32", "versions": "0+", "default": "0x7fffffff", "ignorable": true, + "about": "The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored." }, + { "name": "Topics", "type": "[]FetchTopic", "versions": "0+", + "about": "The topics to fetch.", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID."}, + { "name": "Partitions", "type": "[]FetchPartition", "versions": "0+", + "about": "The partitions to fetch.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "PartitionMaxBytes", "type": "int32", "versions": "0+", + "about": "The maximum bytes to fetch from this partition. 0 when only acknowledgement with no fetching is required. See KIP-74 for cases where this limit may not be honored." }, + { "name": "AcknowledgementBatches", "type": "[]AcknowledgementBatch", "versions": "0+", + "about": "Record batches to acknowledge.", "fields": [ + { "name": "FirstOffset", "type": "int64", "versions": "0+", + "about": "First offset of batch of records to acknowledge."}, + { "name": "LastOffset", "type": "int64", "versions": "0+", + "about": "Last offset (inclusive) of batch of records to acknowledge."}, + { "name": "AcknowledgeTypes", "type": "[]int8", "versions": "0+", + "about": "Array of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject."} + ]} + ]} + ]}, + { "name": "ForgottenTopicsData", "type": "[]ForgottenTopic", "versions": "0+", "ignorable": false, + "about": "The partitions to remove from this share session.", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID."}, + { "name": "Partitions", "type": "[]int32", "versions": "0+", + "about": "The partitions indexes to forget." } + ]} + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareFetchResponse.json b/clients/src/main/resources/common/message/ShareFetchResponse.json new file mode 100644 index 0000000000000..5338e1208a7bc --- /dev/null +++ b/clients/src/main/resources/common/message/ShareFetchResponse.json @@ -0,0 +1,83 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 78, + "type": "response", + "name": "ShareFetchResponse", + "validVersions": "0", + "flexibleVersions": "0+", + // Supported errors for ErrorCode and AcknowledgeErrorCode: + // - GROUP_AUTHORIZATION_FAILED (version 0+) + // - TOPIC_AUTHORIZATION_FAILED (version 0+) + // - SHARE_SESSION_NOT_FOUND (version 0+) + // - INVALID_SHARE_SESSION_EPOCH (version 0+) + // - UNKNOWN_TOPIC_OR_PARTITION (version 0+) + // - NOT_LEADER_OR_FOLLOWER (version 0+) + // - UNKNOWN_TOPIC_ID (version 0+) + // - INVALID_RECORD_STATE (version 0+) - only for AcknowledgeErrorCode + // - KAFKA_STORAGE_ERROR (version 0+) + // - CORRUPT_MESSAGE (version 0+) + // - INVALID_REQUEST (version 0+) + // - UNKNOWN_SERVER_ERROR (version 0+) + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", "ignorable": true, + "about": "The top-level response error code." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The top-level error message, or null if there was no error." }, + { "name": "Responses", "type": "[]ShareFetchableTopicResponse", "versions": "0+", + "about": "The response topics.", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID."}, + { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", + "about": "The topic partitions.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The fetch error code, or 0 if there was no fetch error." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The fetch error message, or null if there was no fetch error." }, + { "name": "AcknowledgeErrorCode", "type": "int16", "versions": "0+", + "about": "The acknowledge error code, or 0 if there was no acknowledge error." }, + { "name": "AcknowledgeErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The acknowledge error message, or null if there was no acknowledge error." }, + { "name": "CurrentLeader", "type": "LeaderIdAndEpoch", "versions": "0+", "fields": [ + { "name": "LeaderId", "type": "int32", "versions": "0+", + "about": "The ID of the current leader or -1 if the leader is unknown." }, + { "name": "LeaderEpoch", "type": "int32", "versions": "0+", + "about": "The latest known leader epoch." } + ]}, + { "name": "Records", "type": "records", "versions": "0+", "nullableVersions": "0+", "about": "The record data."}, + { "name": "AcquiredRecords", "type": "[]AcquiredRecords", "versions": "0+", "about": "The acquired records.", "fields": [ + {"name": "FirstOffset", "type": "int64", "versions": "0+", "about": "The earliest offset in this batch of acquired records."}, + {"name": "LastOffset", "type": "int64", "versions": "0+", "about": "The last offset of this batch of acquired records."}, + {"name": "DeliveryCount", "type": "int16", "versions": "0+", "about": "The delivery count of this batch of acquired records."} + ]} + ]} + ]}, + { "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "0+", + "about": "Endpoints for all current leaders enumerated in PartitionData with error NOT_LEADER_OR_FOLLOWER.", "fields": [ + { "name": "NodeId", "type": "int32", "versions": "0+", + "mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node." }, + { "name": "Host", "type": "string", "versions": "0+", + "about": "The node's hostname." }, + { "name": "Port", "type": "int32", "versions": "0+", + "about": "The node's port." }, + { "name": "Rack", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The rack of the node, or null if it has not been assigned to a rack." } + ]} + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareGroupDescribeRequest.json b/clients/src/main/resources/common/message/ShareGroupDescribeRequest.json new file mode 100644 index 0000000000000..c95790c9b198f --- /dev/null +++ b/clients/src/main/resources/common/message/ShareGroupDescribeRequest.json @@ -0,0 +1,33 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 77, + "type": "request", + "listeners": ["broker"], + "name": "ShareGroupDescribeRequest", + "validVersions": "0", + "flexibleVersions": "0+", + // The ShareGroupDescribeRequest API is added as part of KIP-932 and is still under + // development. Hence, the API is not exposed by default by brokers unless + // explicitly enabled. + "latestVersionUnstable": true, + "fields": [ + { "name": "GroupIds", "type": "[]string", "versions": "0+", "entityType": "groupId", + "about": "The ids of the groups to describe" }, + { "name": "IncludeAuthorizedOperations", "type": "bool", "versions": "0+", + "about": "Whether to include authorized operations." } + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareGroupDescribeResponse.json b/clients/src/main/resources/common/message/ShareGroupDescribeResponse.json new file mode 100644 index 0000000000000..c093b788bfc2f --- /dev/null +++ b/clients/src/main/resources/common/message/ShareGroupDescribeResponse.json @@ -0,0 +1,87 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 77, + "type": "response", + "name": "ShareGroupDescribeResponse", + "validVersions": "0", + "flexibleVersions": "0+", + // Supported errors: + // - GROUP_AUTHORIZATION_FAILED (version 0+) + // - NOT_COORDINATOR (version 0+) + // - COORDINATOR_NOT_AVAILABLE (version 0+) + // - COORDINATOR_LOAD_IN_PROGRESS (version 0+) + // - INVALID_REQUEST (version 0+) + // - INVALID_GROUP_ID (version 0+) + // - GROUP_ID_NOT_FOUND (version 0+) + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Groups", "type": "[]DescribedGroup", "versions": "0+", + "about": "Each described group.", + "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The describe error, or 0 if there was no error." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The top-level error message, or null if there was no error." }, + { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", + "about": "The group ID string." }, + { "name": "GroupState", "type": "string", "versions": "0+", + "about": "The group state string, or the empty string." }, + { "name": "GroupEpoch", "type": "int32", "versions": "0+", + "about": "The group epoch." }, + { "name": "AssignmentEpoch", "type": "int32", "versions": "0+", + "about": "The assignment epoch." }, + { "name": "AssignorName", "type": "string", "versions": "0+", + "about": "The selected assignor." }, + { "name": "Members", "type": "[]Member", "versions": "0+", + "about": "The members.", + "fields": [ + { "name": "MemberId", "type": "string", "versions": "0+", + "about": "The member ID." }, + { "name": "RackId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The member rack ID." }, + { "name": "MemberEpoch", "type": "int32", "versions": "0+", + "about": "The current member epoch." }, + { "name": "ClientId", "type": "string", "versions": "0+", + "about": "The client ID." }, + { "name": "ClientHost", "type": "string", "versions": "0+", + "about": "The client host." }, + { "name": "SubscribedTopicNames", "type": "[]string", "versions": "0+", "entityType": "topicName", + "about": "The subscribed topic names." }, + { "name": "Assignment", "type": "Assignment", "versions": "0+", + "about": "The current assignment." } + ]}, + { "name": "AuthorizedOperations", "type": "int32", "versions": "0+", "default": "-2147483648", + "about": "32-bit bitfield to represent authorized operations for this group." } + ] + } + ], + "commonStructs": [ + { "name": "TopicPartitions", "versions": "0+", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", + "about": "The topic ID." }, + { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]int32", "versions": "0+", + "about": "The partitions." } + ]}, + { "name": "Assignment", "versions": "0+", "fields": [ + { "name": "TopicPartitions", "type": "[]TopicPartitions", "versions": "0+", + "about": "The assigned topic-partitions to the member." } + ]} + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json b/clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json new file mode 100644 index 0000000000000..7d28c116454d3 --- /dev/null +++ b/clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json @@ -0,0 +1,39 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 76, + "type": "request", + "listeners": ["broker"], + "name": "ShareGroupHeartbeatRequest", + "validVersions": "0", + "flexibleVersions": "0+", + // The ShareGroupHeartbeatRequest API is added as part of KIP-932 and is still under + // development. Hence, the API is not exposed by default by brokers unless + // explicitly enabled. + "latestVersionUnstable": true, + "fields": [ + { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", + "about": "The group identifier." }, + { "name": "MemberId", "type": "string", "versions": "0+", + "about": "The member ID generated by the coordinator. The member ID must be kept during the entire lifetime of the member." }, + { "name": "MemberEpoch", "type": "int32", "versions": "0+", + "about": "The current member epoch; 0 to join the group; -1 to leave the group." }, + { "name": "RackId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "null if not provided or if it didn't change since the last heartbeat; the rack ID of consumer otherwise." }, + { "name": "SubscribedTopicNames", "type": "[]string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "null if it didn't change since the last heartbeat; the subscribed topic names otherwise." } + ] +} \ No newline at end of file diff --git a/clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json b/clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json new file mode 100644 index 0000000000000..e692839f29bf9 --- /dev/null +++ b/clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json @@ -0,0 +1,57 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 76, + "type": "response", + "name": "ShareGroupHeartbeatResponse", + "validVersions": "0", + "flexibleVersions": "0+", + // Supported errors: + // - GROUP_AUTHORIZATION_FAILED (version 0+) + // - NOT_COORDINATOR (version 0+) + // - COORDINATOR_NOT_AVAILABLE (version 0+) + // - COORDINATOR_LOAD_IN_PROGRESS (version 0+) + // - INVALID_REQUEST (version 0+) + // - UNKNOWN_MEMBER_ID (version 0+) + // - GROUP_MAX_SIZE_REACHED (version 0+) + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The top-level error code, or 0 if there was no error" }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The top-level error message, or null if there was no error." }, + { "name": "MemberId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The member ID generated by the coordinator. Only provided when the member joins with MemberEpoch == 0." }, + { "name": "MemberEpoch", "type": "int32", "versions": "0+", + "about": "The member epoch." }, + { "name": "HeartbeatIntervalMs", "type": "int32", "versions": "0+", + "about": "The heartbeat interval in milliseconds." }, + { "name": "Assignment", "type": "Assignment", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "null if not provided; the assignment otherwise.", "fields": [ + { "name": "TopicPartitions", "type": "[]TopicPartitions", "versions": "0+", + "about": "The partitions assigned to the member." } + ]} + ], + "commonStructs": [ + { "name": "TopicPartitions", "versions": "0+", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "0+", + "about": "The topic ID." }, + { "name": "Partitions", "type": "[]int32", "versions": "0+", + "about": "The partitions." } + ]} + ] +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java b/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java index 80e5ddfc4d830..76937059ce975 100644 --- a/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java @@ -77,6 +77,11 @@ public void testNoPort() { assertThrows(ConfigException.class, () -> checkWithoutLookup("127.0.0.1")); } + @Test + public void testInvalidPort() { + assertThrows(ConfigException.class, () -> checkWithoutLookup("localhost:70000")); + } + @Test public void testOnlyBadHostname() { assertThrows(ConfigException.class, () -> checkWithoutLookup("some.invalid.hostname.foo.bar.local:9999")); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index eab9f3e5ac12d..66ee724a0e515 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -1032,6 +1032,47 @@ public void testOffsetsForTimes() { verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class)); } + @Test + public void testOffsetsForTimesTimeoutException() { + consumer = newConsumer(); + long timeout = 100; + doThrow(new TimeoutException("Event did not complete in time and was expired by the reaper")) + .when(applicationEventHandler).addAndGet(any()); + + Throwable t = assertThrows( + TimeoutException.class, + () -> consumer.offsetsForTimes(mockTimestampToSearch(), Duration.ofMillis(timeout))); + assertEquals("Failed to get offsets by times in " + timeout + "ms", t.getMessage()); + } + + @Test + public void testBeginningOffsetsTimeoutException() { + consumer = newConsumer(); + long timeout = 100; + doThrow(new TimeoutException("Event did not complete in time and was expired by the reaper")) + .when(applicationEventHandler).addAndGet(any()); + + Throwable t = assertThrows( + TimeoutException.class, + () -> consumer.beginningOffsets(Collections.singleton(new TopicPartition("topic", 5)), + Duration.ofMillis(timeout))); + assertEquals("Failed to get offsets by times in " + timeout + "ms", t.getMessage()); + } + + @Test + public void testEndOffsetsTimeoutException() { + consumer = newConsumer(); + long timeout = 100; + doThrow(new TimeoutException("Event did not complete in time and was expired by the reaper")) + .when(applicationEventHandler).addAndGet(any()); + + Throwable t = assertThrows( + TimeoutException.class, + () -> consumer.endOffsets(Collections.singleton(new TopicPartition("topic", 5)), + Duration.ofMillis(timeout))); + assertEquals("Failed to get offsets by times in " + timeout + "ms", t.getMessage()); + } + // This test ensures same behaviour as the current consumer when offsetsForTimes is called // with 0 timeout. It should return map with all requested partitions as keys, with null // OffsetAndTimestamp as value. diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CompletedFetchTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CompletedFetchTest.java index 803c2f14cb275..5b437c2fb5e0e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CompletedFetchTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CompletedFetchTest.java @@ -23,6 +23,8 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.errors.RecordDeserializationException; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.ApiKeys; @@ -41,6 +43,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import org.junit.jupiter.api.Test; import java.nio.ByteBuffer; @@ -48,7 +51,9 @@ import java.util.List; import java.util.UUID; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; public class CompletedFetchTest { @@ -161,6 +166,10 @@ public void testCorruptedMessage() { final UUIDSerializer serializer = new UUIDSerializer()) { builder.append(new SimpleRecord(serializer.serialize(TOPIC_NAME, UUID.randomUUID()))); builder.append(0L, "key".getBytes(), "value".getBytes()); + builder.append(new SimpleRecord(serializer.serialize(TOPIC_NAME, UUID.randomUUID()))); + Headers headers = new RecordHeaders(); + headers.add("hkey", "hvalue".getBytes()); + builder.append(10L, serializer.serialize("key", UUID.randomUUID()), "otherValue".getBytes(), headers.toArray()); Records records = builder.build(); FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() @@ -176,8 +185,29 @@ public void testCorruptedMessage() { completedFetch.fetchRecords(fetchConfig, deserializers, 10); - assertThrows(RecordDeserializationException.class, + RecordDeserializationException thrown = assertThrows(RecordDeserializationException.class, () -> completedFetch.fetchRecords(fetchConfig, deserializers, 10)); + assertEquals(RecordDeserializationException.DeserializationExceptionOrigin.KEY, thrown.origin()); + assertEquals(1, thrown.offset()); + assertEquals(TOPIC_NAME, thrown.topicPartition().topic()); + assertEquals(0, thrown.topicPartition().partition()); + assertEquals(0, thrown.timestamp()); + assertArrayEquals("key".getBytes(), Utils.toNullableArray(thrown.keyBuffer())); + assertArrayEquals("value".getBytes(), Utils.toNullableArray(thrown.valueBuffer())); + assertEquals(0, thrown.headers().toArray().length); + + CompletedFetch completedFetch2 = newCompletedFetch(2, partitionData); + completedFetch2.fetchRecords(fetchConfig, deserializers, 10); + RecordDeserializationException valueThrown = assertThrows(RecordDeserializationException.class, + () -> completedFetch2.fetchRecords(fetchConfig, deserializers, 10)); + assertEquals(RecordDeserializationException.DeserializationExceptionOrigin.VALUE, valueThrown.origin()); + assertEquals(3, valueThrown.offset()); + assertEquals(TOPIC_NAME, valueThrown.topicPartition().topic()); + assertEquals(0, valueThrown.topicPartition().partition()); + assertEquals(10L, valueThrown.timestamp()); + assertNotNull(valueThrown.keyBuffer()); + assertArrayEquals("otherValue".getBytes(), Utils.toNullableArray(valueThrown.valueBuffer())); + assertEquals(headers, valueThrown.headers()); } } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index f7a73dbae8a9b..f63dd55754a76 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager.HeartbeatState; import org.apache.kafka.clients.consumer.internals.MembershipManager.LocalAssignment; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.Uuid; @@ -51,6 +52,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; import java.util.Arrays; import java.util.Collection; @@ -275,7 +277,7 @@ public void testHeartbeatNotSentIfAnotherOneInFlight() { result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a " + "previous one is in-flight"); - + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); result = heartbeatRequestManager.poll(time.milliseconds()); assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent when the " + @@ -483,8 +485,7 @@ public void testHeartbeatResponseOnErrorHandling(final Errors error, final boole break; default: if (isFatal) { - // The memberStateManager should have stopped heartbeat at this point - ensureFatalError(); + ensureFatalError(error); } else { verify(backgroundEventHandler, never()).add(any()); assertNextHeartbeatTiming(0); @@ -751,6 +752,25 @@ public void testFencedMemberStopHeartbeatUntilItReleasesAssignmentToRejoin() { assertEquals(1, result.unsentRequests.size(), "Fenced member should resume heartbeat after transitioning to JOINING"); } + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) + public void testSendingLeaveGroupHeartbeatWhenPreviousOneInFlight(final short version) { + mockStableMember(); + time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, result.unsentRequests.size(), "No heartbeat should be sent while a previous one is in-flight"); + + membershipManager.leaveGroup(); + + ConsumerGroupHeartbeatRequest heartbeatToLeave = getHeartbeatRequest(heartbeatRequestManager, version); + assertEquals(ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH, heartbeatToLeave.data().memberEpoch()); + + NetworkClientDelegate.PollResult pollAgain = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, pollAgain.unsentRequests.size()); + } + private void assertHeartbeat(HeartbeatRequestManager hrm, int nextPollMs) { NetworkClientDelegate.PollResult pollResult = hrm.poll(time.milliseconds()); assertEquals(1, pollResult.unsentRequests.size()); @@ -781,9 +801,15 @@ private void mockStableMember() { assertEquals(MemberState.STABLE, membershipManager.state()); } - private void ensureFatalError() { + private void ensureFatalError(Errors expectedError) { verify(membershipManager).transitionToFatal(); - verify(backgroundEventHandler).add(any()); + + final ArgumentCaptor errorEventArgumentCaptor = ArgumentCaptor.forClass(ErrorEvent.class); + verify(backgroundEventHandler).add(errorEventArgumentCaptor.capture()); + ErrorEvent errorEvent = errorEventArgumentCaptor.getValue(); + assertInstanceOf(expectedError.exception().getClass(), errorEvent.error(), + "The fatal error propagated to the app thread does not match the error received in the heartbeat response."); + ensureHeartbeatStopped(); } @@ -808,6 +834,7 @@ private static Collection errorProvider() { Arguments.of(Errors.UNSUPPORTED_ASSIGNOR, true), Arguments.of(Errors.UNSUPPORTED_VERSION, true), Arguments.of(Errors.UNRELEASED_INSTANCE_ID, true), + Arguments.of(Errors.FENCED_INSTANCE_ID, true), Arguments.of(Errors.GROUP_MAX_SIZE_REACHED, true)); } diff --git a/clients/src/test/java/org/apache/kafka/common/ClusterTest.java b/clients/src/test/java/org/apache/kafka/common/ClusterTest.java index 4c6db86d6a722..f209b48f7171c 100644 --- a/clients/src/test/java/org/apache/kafka/common/ClusterTest.java +++ b/clients/src/test/java/org/apache/kafka/common/ClusterTest.java @@ -21,12 +21,15 @@ import java.net.InetSocketAddress; import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import static java.util.Arrays.asList; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertThrows; public class ClusterTest { @@ -89,4 +92,74 @@ public void testReturnUnmodifiableCollections() { new PartitionInfo(TOPIC_B, 2, NODES[1], NODES, NODES))); } + @Test + public void testNotEquals() { + String clusterId1 = "clusterId1"; + String clusterId2 = "clusterId2"; + Node node0 = new Node(0, "host0", 100); + Node node1 = new Node(1, "host1", 100); + Set partitions1 = Collections.singleton(new PartitionInfo("topic1", 0, node0, new Node[]{node0, node1}, new Node[]{node0})); + Set partitions2 = Collections.singleton(new PartitionInfo("topic2", 0, node0, new Node[]{node1, node0}, new Node[]{node1})); + Set unauthorizedTopics1 = Collections.singleton("topic1"); + Set unauthorizedTopics2 = Collections.singleton("topic2"); + Set invalidTopics1 = Collections.singleton("topic1"); + Set invalidTopics2 = Collections.singleton("topic2"); + Set internalTopics1 = Collections.singleton("topic3"); + Set internalTopics2 = Collections.singleton("topic4"); + Node controller1 = new Node(2, "host2", 100); + Node controller2 = new Node(3, "host3", 100); + Map topicIds1 = Collections.singletonMap("topic1", Uuid.randomUuid()); + Map topicIds2 = Collections.singletonMap("topic2", Uuid.randomUuid()); + + Cluster cluster1 = new Cluster(clusterId1, Collections.singletonList(node0), partitions1, + unauthorizedTopics1, invalidTopics1, internalTopics1, controller1, topicIds1); + Cluster differentTopicIds = new Cluster(clusterId1, Collections.singletonList(node0), partitions1, + unauthorizedTopics1, invalidTopics1, internalTopics1, controller1, topicIds2); + Cluster differentController = new Cluster(clusterId1, Collections.singletonList(node0), partitions1, + unauthorizedTopics1, invalidTopics1, internalTopics1, controller2, topicIds1); + Cluster differentInternalTopics = new Cluster(clusterId1, Collections.singletonList(node0), partitions1, + unauthorizedTopics1, invalidTopics1, internalTopics2, controller1, topicIds1); + Cluster differentInvalidTopics = new Cluster(clusterId1, Collections.singletonList(node0), partitions1, + unauthorizedTopics1, invalidTopics2, internalTopics1, controller1, topicIds1); + Cluster differentUnauthorizedTopics = new Cluster(clusterId1, Collections.singletonList(node0), partitions1, + unauthorizedTopics2, invalidTopics1, internalTopics1, controller1, topicIds1); + Cluster differentPartitions = new Cluster(clusterId1, Collections.singletonList(node0), partitions2, + unauthorizedTopics1, invalidTopics1, internalTopics1, controller1, topicIds1); + Cluster differentNodes = new Cluster(clusterId1, Arrays.asList(node0, node1), partitions1, + unauthorizedTopics1, invalidTopics1, internalTopics1, controller1, topicIds1); + Cluster differentClusterId = new Cluster(clusterId2, Collections.singletonList(node0), partitions1, + unauthorizedTopics1, invalidTopics1, internalTopics1, controller1, topicIds1); + + assertNotEquals(cluster1, differentTopicIds); + assertNotEquals(cluster1, differentController); + assertNotEquals(cluster1, differentInternalTopics); + assertNotEquals(cluster1, differentInvalidTopics); + assertNotEquals(cluster1, differentUnauthorizedTopics); + assertNotEquals(cluster1, differentPartitions); + assertNotEquals(cluster1, differentNodes); + assertNotEquals(cluster1, differentClusterId); + } + + @Test + public void testEquals() { + String clusterId1 = "clusterId1"; + Node node1 = new Node(1, "host0", 100); + Node node1duplicate = new Node(1, "host0", 100); + Set partitions1 = Collections.singleton(new PartitionInfo("topic1", 0, node1, new Node[]{node1}, new Node[]{node1})); + Set partitions1duplicate = Collections.singleton(new PartitionInfo("topic1", 0, node1duplicate, new Node[]{node1duplicate}, new Node[]{node1duplicate})); + Set unauthorizedTopics1 = Collections.singleton("topic1"); + Set invalidTopics1 = Collections.singleton("topic1"); + Set internalTopics1 = Collections.singleton("topic3"); + Node controller1 = new Node(2, "host0", 100); + Node controller1duplicate = new Node(2, "host0", 100); + Uuid topicId1 = Uuid.randomUuid(); + Map topicIds1 = Collections.singletonMap("topic1", topicId1); + Map topicIds1duplicate = Collections.singletonMap("topic1", topicId1); + + Cluster cluster1 = new Cluster(clusterId1, Collections.singletonList(node1), partitions1, unauthorizedTopics1, + invalidTopics1, internalTopics1, controller1, topicIds1); + Cluster cluster1duplicate = new Cluster(clusterId1, Collections.singletonList(node1duplicate), partitions1duplicate, + unauthorizedTopics1, invalidTopics1, internalTopics1, controller1duplicate, topicIds1duplicate); + assertEquals(cluster1, cluster1duplicate); + } } 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 f514281abf314..a4289d8afbd23 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 @@ -336,17 +336,6 @@ public void testRemoveMetric() { assertEquals(size, metrics.metrics().size()); } - @Test - public void testEventWindowing() { - WindowedCount count = new WindowedCount(); - MetricConfig config = new MetricConfig().eventWindow(1).samples(2); - count.record(config, 1.0, time.milliseconds()); - count.record(config, 1.0, time.milliseconds()); - assertEquals(2.0, count.measure(config, time.milliseconds()), EPS); - count.record(config, 1.0, time.milliseconds()); // first event times out - assertEquals(2.0, count.measure(config, time.milliseconds()), EPS); - } - @Test public void testTimeWindowing() { WindowedCount count = new WindowedCount(); @@ -475,28 +464,13 @@ public void testPercentiles() { Metric p50 = this.metrics.metrics().get(metrics.metricName("test.p50", "grp1")); Metric p75 = this.metrics.metrics().get(metrics.metricName("test.p75", "grp1")); - // record two windows worth of sequential values - for (int i = 0; i < buckets; i++) - sensor.record(i); - - assertEquals(25, (Double) p25.metricValue(), 1.0); - assertEquals(50, (Double) p50.metricValue(), 1.0); - assertEquals(75, (Double) p75.metricValue(), 1.0); - - for (int i = 0; i < buckets; i++) - sensor.record(0.0); - - assertEquals(0.0, (Double) p25.metricValue(), 1.0); - assertEquals(0.0, (Double) p50.metricValue(), 1.0); - assertEquals(0.0, (Double) p75.metricValue(), 1.0); - - // record two more windows worth of sequential values + // record 100 sequential values for (int i = 0; i < buckets; i++) sensor.record(i); - assertEquals(25, (Double) p25.metricValue(), 1.0); - assertEquals(50, (Double) p50.metricValue(), 1.0); - assertEquals(75, (Double) p75.metricValue(), 1.0); + assertEquals(25, (Double) p25.metricValue()); + assertEquals(50, (Double) p50.metricValue()); + assertEquals(75, (Double) p75.metricValue()); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/stats/FrequenciesTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/stats/FrequenciesTest.java index 344ade22a66ed..ba3067d993d9f 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/stats/FrequenciesTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/stats/FrequenciesTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.common.metrics.stats; -import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.CompoundStat.NamedMeasurable; import org.apache.kafka.common.metrics.JmxReporter; @@ -73,14 +72,14 @@ public void testMoreFrequencyParametersThanBuckets() { } @Test - public void testBooleanFrequencies() { - MetricName metricTrue = name("true"); - MetricName metricFalse = name("false"); + public void testBooleanFrequenciesStrategy1() { + MetricName metricTrue = metricName("true"); + MetricName metricFalse = metricName("false"); Frequencies frequencies = Frequencies.forBooleanValues(metricFalse, metricTrue); final NamedMeasurable falseMetric = frequencies.stats().get(0); final NamedMeasurable trueMetric = frequencies.stats().get(1); - // Record 2 windows worth of values + // Record 25 "false" and 75 "true" for (int i = 0; i != 25; ++i) { frequencies.record(config, 0.0, time.milliseconds()); } @@ -89,8 +88,17 @@ public void testBooleanFrequencies() { } assertEquals(0.25, falseMetric.stat().measure(config, time.milliseconds()), DELTA); assertEquals(0.75, trueMetric.stat().measure(config, time.milliseconds()), DELTA); + } + + @Test + public void testBooleanFrequenciesStrategy2() { + MetricName metricTrue = metricName("true"); + MetricName metricFalse = metricName("false"); + Frequencies frequencies = Frequencies.forBooleanValues(metricFalse, metricTrue); + final NamedMeasurable falseMetric = frequencies.stats().get(0); + final NamedMeasurable trueMetric = frequencies.stats().get(1); - // Record 2 more windows worth of values + // Record 40 "false" and 60 "true" for (int i = 0; i != 40; ++i) { frequencies.record(config, 0.0, time.milliseconds()); } @@ -102,58 +110,69 @@ public void testBooleanFrequencies() { } @Test - public void testUseWithMetrics() { - MetricName name1 = name("1"); - MetricName name2 = name("2"); - MetricName name3 = name("3"); - MetricName name4 = name("4"); - Frequencies frequencies = new Frequencies(4, 1.0, 4.0, - new Frequency(name1, 1.0), - new Frequency(name2, 2.0), - new Frequency(name3, 3.0), - new Frequency(name4, 4.0)); + public void testWithMetricsStrategy1() { + Frequencies frequencies = new Frequencies(4, 1.0, 4.0, freq("1", 1.0), + freq("2", 2.0), freq("3", 3.0), freq("4", 4.0)); Sensor sensor = metrics.sensor("test", config); sensor.add(frequencies); - Metric metric1 = this.metrics.metrics().get(name1); - Metric metric2 = this.metrics.metrics().get(name2); - Metric metric3 = this.metrics.metrics().get(name3); - Metric metric4 = this.metrics.metrics().get(name4); - // Record 2 windows worth of values - for (int i = 0; i != 100; ++i) { + // Record 100 events uniformly between all buckets + for (int i = 0; i < 100; ++i) { frequencies.record(config, i % 4 + 1, time.milliseconds()); } - assertEquals(0.25, (Double) metric1.metricValue(), DELTA); - assertEquals(0.25, (Double) metric2.metricValue(), DELTA); - assertEquals(0.25, (Double) metric3.metricValue(), DELTA); - assertEquals(0.25, (Double) metric4.metricValue(), DELTA); + assertEquals(0.25, metricValue("1"), DELTA); + assertEquals(0.25, metricValue("2"), DELTA); + assertEquals(0.25, metricValue("3"), DELTA); + assertEquals(0.25, metricValue("4"), DELTA); + } + + @Test + public void testWithMetricsStrategy2() { + Frequencies frequencies = new Frequencies(4, 1.0, 4.0, freq("1", 1.0), + freq("2", 2.0), freq("3", 3.0), freq("4", 4.0)); + Sensor sensor = metrics.sensor("test", config); + sensor.add(frequencies); + + // Record 100 events half-half between 1st and 2nd buckets + for (int i = 0; i < 100; ++i) { + frequencies.record(config, i % 2 + 1, time.milliseconds()); + } + assertEquals(0.50, metricValue("1"), DELTA); + assertEquals(0.50, metricValue("2"), DELTA); + assertEquals(0.00, metricValue("3"), DELTA); + assertEquals(0.00, metricValue("4"), DELTA); + } - // Record 2 windows worth of values - for (int i = 0; i != 100; ++i) { + @Test + public void testWithMetricsStrategy3() { + Frequencies frequencies = new Frequencies(4, 1.0, 4.0, freq("1", 1.0), + freq("2", 2.0), freq("3", 3.0), freq("4", 4.0)); + Sensor sensor = metrics.sensor("test", config); + sensor.add(frequencies); + + // Record 50 events half-half between 1st and 2nd buckets + for (int i = 0; i < 50; ++i) { frequencies.record(config, i % 2 + 1, time.milliseconds()); } - assertEquals(0.50, (Double) metric1.metricValue(), DELTA); - assertEquals(0.50, (Double) metric2.metricValue(), DELTA); - assertEquals(0.00, (Double) metric3.metricValue(), DELTA); - assertEquals(0.00, (Double) metric4.metricValue(), DELTA); - - // Record 1 window worth of values to overlap with the last window - // that is half 1.0 and half 2.0 - for (int i = 0; i != 50; ++i) { + // Record 50 events to 4th bucket + for (int i = 0; i < 50; ++i) { frequencies.record(config, 4.0, time.milliseconds()); } - assertEquals(0.25, (Double) metric1.metricValue(), DELTA); - assertEquals(0.25, (Double) metric2.metricValue(), DELTA); - assertEquals(0.00, (Double) metric3.metricValue(), DELTA); - assertEquals(0.50, (Double) metric4.metricValue(), DELTA); + assertEquals(0.25, metricValue("1"), DELTA); + assertEquals(0.25, metricValue("2"), DELTA); + assertEquals(0.00, metricValue("3"), DELTA); + assertEquals(0.50, metricValue("4"), DELTA); } - protected MetricName name(String metricName) { - return new MetricName(metricName, "group-id", "desc", Collections.emptyMap()); + private MetricName metricName(String name) { + return new MetricName(name, "group-id", "desc", Collections.emptyMap()); } - protected Frequency freq(String name, double value) { - return new Frequency(name(name), value); + private Frequency freq(String name, double value) { + return new Frequency(metricName(name), value); } + private double metricValue(String name) { + return (double) metrics.metrics().get(metricName(name)).metricValue(); + } } 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 index 04c5ca1292f98..2e8662f1f96c9 100644 --- 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 @@ -21,23 +21,28 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; +import java.util.Arrays; +import java.util.List; import java.util.concurrent.TimeUnit; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; public class RateTest { private static final double EPS = 0.000001; - private Rate r; - private Time timeClock; + private Rate rate; + private Time time; @BeforeEach public void setup() { - r = new Rate(); - timeClock = new MockTime(); + rate = new Rate(); + time = new MockTime(); } // Tests the scenario where the recording and measurement is done before the window for first sample finishes @@ -48,12 +53,12 @@ public void testRateWithNoPriorAvailableSamples(int numSample, int sampleWindowS 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()); + rate.record(config, sampleValue, time.milliseconds()); // forward time till almost the end of window final long measurementTime = TimeUnit.SECONDS.toMillis(sampleWindowSizeSec) - 1; - timeClock.sleep(measurementTime); + time.sleep(measurementTime); // calculate rate at almost the end of window - final double observedRate = r.measure(config, timeClock.milliseconds()); + final double observedRate = rate.measure(config, time.milliseconds()); assertFalse(Double.isNaN(observedRate)); // In a scenario where sufficient number of samples is not available yet, the rate calculation algorithm assumes @@ -64,4 +69,34 @@ public void testRateWithNoPriorAvailableSamples(int numSample, int sampleWindowS double expectedRatePerSec = sampleValue / windowSize; assertEquals(expectedRatePerSec, observedRate, EPS); } + + // Record an event every 100 ms on average, moving some 1 ms back or forth for fine-grained + // window control. The expected rate, hence, is 10-11 events/sec depending on the moment of + // measurement. Start assertions from the second window. This test covers the case where a + // sample window partially overlaps with the monitored window. + @Test + public void testRateIsConsistentAfterTheFirstWindow() { + MetricConfig config = new MetricConfig().timeWindow(1, SECONDS).samples(2); + List steps = Arrays.asList(0, 99, 100, 100, 100, 100, 100, 100, 100, 100, 100); + + // start the first window and record events at 0,99,199,...,999 ms + for (int stepMs : steps) { + time.sleep(stepMs); + rate.record(config, 1, time.milliseconds()); + } + + // making a gap of 100 ms between windows + time.sleep(101); + + // start the second window and record events at 0,99,199,...,999 ms + for (int stepMs : steps) { + time.sleep(stepMs); + rate.record(config, 1, time.milliseconds()); + double observedRate = rate.measure(config, time.milliseconds()); + assertTrue(10 <= observedRate && observedRate <= 11); + // make sure measurements are repeatable with the same timestamp + double measuredAgain = rate.measure(config, time.milliseconds()); + assertEquals(observedRate, measuredAgain); + } + } } diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/stats/SampledStatTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/stats/SampledStatTest.java new file mode 100644 index 0000000000000..7b45131b9b481 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/metrics/stats/SampledStatTest.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.metrics.stats; + +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.junit.jupiter.api.Assertions.assertEquals; + +class SampledStatTest { + + private SampledStat stat; + private Time time; + + @BeforeEach + public void setup() { + stat = new SampleCount(); + time = new MockTime(); + } + + @Test + @DisplayName("Sample should be purged if doesn't overlap the window") + public void testSampleIsPurgedIfDoesntOverlap() { + MetricConfig config = new MetricConfig().timeWindow(1, SECONDS).samples(2); + + // Monitored window: 2s. Complete a sample and wait 2.5s after. + completeSample(config); + time.sleep(2500); + + double numSamples = stat.measure(config, time.milliseconds()); + assertEquals(0, numSamples); + } + + @Test + @DisplayName("Sample should be kept if overlaps the window") + public void testSampleIsKeptIfOverlaps() { + MetricConfig config = new MetricConfig().timeWindow(1, SECONDS).samples(2); + + // Monitored window: 2s. Complete a sample and wait 1.5s after. + completeSample(config); + time.sleep(1500); + + double numSamples = stat.measure(config, time.milliseconds()); + assertEquals(1, numSamples); + } + + @Test + @DisplayName("Sample should be kept if overlaps the window and is n+1") + public void testSampleIsKeptIfOverlapsAndExtra() { + MetricConfig config = new MetricConfig().timeWindow(1, SECONDS).samples(2); + + // Monitored window: 2s. Create 2 samples with gaps in between and + // take a measurement at 2.2s from the start. + completeSample(config); + time.sleep(100); + completeSample(config); + time.sleep(100); + stat.record(config, 1, time.milliseconds()); + + double numSamples = stat.measure(config, time.milliseconds()); + assertEquals(3, numSamples); + } + + // Creates a sample with events at the start and at the end. Positions clock at the end. + private void completeSample(MetricConfig config) { + stat.record(config, 1, time.milliseconds()); + time.sleep(config.timeWindowMs() - 1); + stat.record(config, 1, time.milliseconds()); + time.sleep(1); + } + + // measure() of this impl returns the number of samples + static class SampleCount extends SampledStat { + + SampleCount() { + super(0); + } + + @Override + protected void update(Sample sample, MetricConfig config, double value, long timeMs) { + sample.value = 1; + } + + @Override + public double combine(List samples, MetricConfig config, long now) { + return samples.stream().mapToDouble(s -> s.value).sum(); + } + } +} 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 512a7cea76681..82487bd418429 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 @@ -20,6 +20,7 @@ import org.apache.kafka.common.ElectionType; import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.Node; +import org.apache.kafka.common.ShareGroupState; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.Uuid; @@ -210,6 +211,14 @@ import org.apache.kafka.common.message.SaslAuthenticateResponseData; import org.apache.kafka.common.message.SaslHandshakeRequestData; import org.apache.kafka.common.message.SaslHandshakeResponseData; +import org.apache.kafka.common.message.ShareAcknowledgeRequestData; +import org.apache.kafka.common.message.ShareAcknowledgeResponseData; +import org.apache.kafka.common.message.ShareFetchRequestData; +import org.apache.kafka.common.message.ShareFetchResponseData; +import org.apache.kafka.common.message.ShareGroupDescribeRequestData; +import org.apache.kafka.common.message.ShareGroupDescribeResponseData; +import org.apache.kafka.common.message.ShareGroupHeartbeatRequestData; +import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData; import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionState; import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaTopicState; import org.apache.kafka.common.message.StopReplicaResponseData; @@ -1001,6 +1010,10 @@ public void testErrorCountsIncludesNone() { assertEquals(1, createTxnOffsetCommitResponse().errorCounts().get(Errors.NONE)); assertEquals(1, createUpdateMetadataResponse().errorCounts().get(Errors.NONE)); assertEquals(1, createWriteTxnMarkersResponse().errorCounts().get(Errors.NONE)); + assertEquals(1, createShareGroupHeartbeatResponse().errorCounts().get(Errors.NONE)); + assertEquals(1, createShareGroupDescribeResponse().errorCounts().get(Errors.NONE)); + assertEquals(2, createShareFetchResponse().errorCounts().get(Errors.NONE)); + assertEquals(2, createShareAcknowledgeResponse().errorCounts().get(Errors.NONE)); } private AbstractRequest getRequest(ApiKeys apikey, short version) { @@ -1081,6 +1094,10 @@ private AbstractRequest getRequest(ApiKeys apikey, short version) { case ASSIGN_REPLICAS_TO_DIRS: return createAssignReplicasToDirsRequest(version); case LIST_CLIENT_METRICS_RESOURCES: return createListClientMetricsResourcesRequest(version); case DESCRIBE_TOPIC_PARTITIONS: return createDescribeTopicPartitionsRequest(version); + case SHARE_GROUP_HEARTBEAT: return createShareGroupHeartbeatRequest(version); + case SHARE_GROUP_DESCRIBE: return createShareGroupDescribeRequest(version); + case SHARE_FETCH: return createShareFetchRequest(version); + case SHARE_ACKNOWLEDGE: return createShareAcknowledgeRequest(version); default: throw new IllegalArgumentException("Unknown API key " + apikey); } } @@ -1163,6 +1180,10 @@ private AbstractResponse getResponse(ApiKeys apikey, short version) { case ASSIGN_REPLICAS_TO_DIRS: return createAssignReplicasToDirsResponse(); case LIST_CLIENT_METRICS_RESOURCES: return createListClientMetricsResourcesResponse(); case DESCRIBE_TOPIC_PARTITIONS: return createDescribeTopicPartitionsResponse(); + case SHARE_GROUP_HEARTBEAT: return createShareGroupHeartbeatResponse(); + case SHARE_GROUP_DESCRIBE: return createShareGroupDescribeResponse(); + case SHARE_FETCH: return createShareFetchResponse(); + case SHARE_ACKNOWLEDGE: return createShareAcknowledgeResponse(); default: throw new IllegalArgumentException("Unknown API key " + apikey); } } @@ -1330,6 +1351,114 @@ private ConsumerGroupHeartbeatResponse createConsumerGroupHeartbeatResponse() { return new ConsumerGroupHeartbeatResponse(data); } + private ShareGroupHeartbeatRequest createShareGroupHeartbeatRequest(short version) { + ShareGroupHeartbeatRequestData data = new ShareGroupHeartbeatRequestData() + .setGroupId("group") + .setMemberId("memberid") + .setMemberEpoch(10) + .setRackId("rackid") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")); + return new ShareGroupHeartbeatRequest.Builder(data).build(version); + } + + private ShareGroupHeartbeatResponse createShareGroupHeartbeatResponse() { + ShareGroupHeartbeatResponseData data = new ShareGroupHeartbeatResponseData() + .setErrorCode(Errors.NONE.code()) + .setThrottleTimeMs(1000) + .setMemberId("memberid") + .setMemberEpoch(11) + .setAssignment(new ShareGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(Arrays.asList( + new ShareGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(Uuid.randomUuid()) + .setPartitions(Arrays.asList(0, 1, 2)), + new ShareGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(Uuid.randomUuid()) + .setPartitions(Arrays.asList(3, 4, 5)) + )) + ); + return new ShareGroupHeartbeatResponse(data); + } + + private ShareGroupDescribeRequest createShareGroupDescribeRequest(short version) { + ShareGroupDescribeRequestData data = new ShareGroupDescribeRequestData() + .setGroupIds(Collections.singletonList("group")) + .setIncludeAuthorizedOperations(false); + return new ShareGroupDescribeRequest.Builder(data).build(version); + } + + private ShareGroupDescribeResponse createShareGroupDescribeResponse() { + ShareGroupDescribeResponseData data = new ShareGroupDescribeResponseData() + .setGroups(Collections.singletonList( + new ShareGroupDescribeResponseData.DescribedGroup() + .setGroupId("group") + .setErrorCode((short) 0) + .setErrorMessage(Errors.forCode((short) 0).message()) + .setGroupState(ShareGroupState.EMPTY.toString()) + .setMembers(new ArrayList<>(0)) + )) + .setThrottleTimeMs(1000); + return new ShareGroupDescribeResponse(data); + } + + private ShareFetchRequest createShareFetchRequest(short version) { + ShareFetchRequestData data = new ShareFetchRequestData() + .setGroupId("group") + .setMemberId(Uuid.randomUuid().toString()) + .setTopics(singletonList(new ShareFetchRequestData.FetchTopic() + .setTopicId(Uuid.randomUuid()) + .setPartitions(singletonList(new ShareFetchRequestData.FetchPartition() + .setPartitionIndex(0))))); + return new ShareFetchRequest.Builder(data).build(version); + } + + private ShareFetchResponse createShareFetchResponse() { + ShareFetchResponseData data = new ShareFetchResponseData(); + MemoryRecords records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("blah".getBytes())); + ShareFetchResponseData.PartitionData partition = new ShareFetchResponseData.PartitionData() + .setPartitionIndex(0) + .setErrorCode(Errors.NONE.code()) + .setRecords(records) + .setAcquiredRecords(singletonList(new ShareFetchResponseData.AcquiredRecords() + .setFirstOffset(0) + .setLastOffset(0) + .setDeliveryCount((short) 1))); + ShareFetchResponseData.ShareFetchableTopicResponse response = new ShareFetchResponseData.ShareFetchableTopicResponse() + .setTopicId(Uuid.randomUuid()) + .setPartitions(singletonList(partition)); + + data.setResponses(singletonList(response)); + data.setThrottleTimeMs(345); + data.setErrorCode(Errors.NONE.code()); + return new ShareFetchResponse(data); + } + + private ShareAcknowledgeRequest createShareAcknowledgeRequest(short version) { + ShareAcknowledgeRequestData data = new ShareAcknowledgeRequestData() + .setMemberId(Uuid.randomUuid().toString()) + .setTopics(singletonList(new ShareAcknowledgeRequestData.AcknowledgeTopic() + .setTopicId(Uuid.randomUuid()) + .setPartitions(singletonList(new ShareAcknowledgeRequestData.AcknowledgePartition() + .setPartitionIndex(0) + .setAcknowledgementBatches(singletonList(new ShareAcknowledgeRequestData.AcknowledgementBatch() + .setFirstOffset(0) + .setLastOffset(0) + .setAcknowledgeTypes(Collections.singletonList((byte) 0)))))))); + return new ShareAcknowledgeRequest.Builder(data).build(version); + } + + private ShareAcknowledgeResponse createShareAcknowledgeResponse() { + ShareAcknowledgeResponseData data = new ShareAcknowledgeResponseData(); + data.setResponses(singletonList(new ShareAcknowledgeResponseData.ShareAcknowledgeTopicResponse() + .setTopicId(Uuid.randomUuid()) + .setPartitions(singletonList(new ShareAcknowledgeResponseData.PartitionData() + .setPartitionIndex(0) + .setErrorCode(Errors.NONE.code()))))); + data.setThrottleTimeMs(345); + data.setErrorCode(Errors.NONE.code()); + return new ShareAcknowledgeResponse(data); + } + private ControllerRegistrationRequest createControllerRegistrationRequest(short version) { ControllerRegistrationRequestData data = new ControllerRegistrationRequestData(). setControllerId(3). 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 197151f5fbc20..97ebe6aaaa088 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 @@ -31,12 +31,12 @@ public class TestSecurityConfig extends AbstractConfig { .define(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, Type.LIST, BrokerSecurityConfigs.DEFAULT_SASL_ENABLED_MECHANISMS, Importance.MEDIUM, BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_DOC) - .define(BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS, Type.CLASS, + .define(BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, Type.CLASS, null, Importance.MEDIUM, BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_DOC) .define(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Type.CLASS, null, Importance.MEDIUM, BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DOC) - .define(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS, Type.LONG, 0L, Importance.MEDIUM, + .define(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, 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) diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java index 477c6283bb498..dc2513e4fc1e0 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java @@ -187,7 +187,7 @@ private TestJaasConfig configureMechanisms(String clientMechanism, List saslClientConfigs.put(SaslConfigs.SASL_MECHANISM, clientMechanism); saslServerConfigs.put(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, serverMechanisms); if (serverMechanisms.contains("DIGEST-MD5")) { - saslServerConfigs.put("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS, + saslServerConfigs.put("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, TestDigestLoginModule.DigestServerCallbackHandler.class.getName()); } return TestJaasConfig.createConfiguration(clientMechanism, serverMechanisms); 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 7063bbe698fff..0b5e172116c9b 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 @@ -348,7 +348,7 @@ public void testClientExceptionDoesNotContainSensitiveData() throws Exception { TestJaasConfig jaasConfig = configureMechanisms("SCRAM-SHA-256", Collections.singletonList("SCRAM-SHA-256")); jaasConfig.createOrUpdateEntry(TestJaasConfig.LOGIN_CONTEXT_SERVER, PlainLoginModule.class.getName(), new HashMap<>()); String callbackPrefix = ListenerName.forSecurityProtocol(securityProtocol).saslMechanismConfigPrefix("SCRAM-SHA-256"); - saslServerConfigs.put(callbackPrefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS, + saslServerConfigs.put(callbackPrefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, InvalidScramServerCallbackHandler.class.getName()); server = createEchoServer(securityProtocol); @@ -645,7 +645,7 @@ public void testTokenReauthenticationOverSaslScram() throws Exception { jaasConfig.createOrUpdateEntry(TestJaasConfig.LOGIN_CONTEXT_CLIENT, ScramLoginModule.class.getName(), options); // ensure re-authentication based on token expiry rather than a default value - saslServerConfigs.put(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS, Long.MAX_VALUE); + saslServerConfigs.put(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, Long.MAX_VALUE); /* * create a token cache that adjusts the token expiration dynamically so that * the first time the expiry is read during authentication we use it to define a @@ -1091,7 +1091,7 @@ public void testServerAuthenticateCallbackHandler() throws Exception { TestJaasConfig jaasConfig = configureMechanisms("PLAIN", Collections.singletonList("PLAIN")); jaasConfig.createOrUpdateEntry(TestJaasConfig.LOGIN_CONTEXT_SERVER, PlainLoginModule.class.getName(), new HashMap<>()); String callbackPrefix = ListenerName.forSecurityProtocol(securityProtocol).saslMechanismConfigPrefix("PLAIN"); - saslServerConfigs.put(callbackPrefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS, + saslServerConfigs.put(callbackPrefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, TestServerCallbackHandler.class.getName()); server = createEchoServer(securityProtocol); @@ -1116,20 +1116,20 @@ public void testAuthenticateCallbackHandlerMechanisms() throws Exception { TestJaasConfig jaasConfig = configureMechanisms("DIGEST-MD5", Arrays.asList("DIGEST-MD5", "PLAIN")); // Connections should fail using the digest callback handler if listener.mechanism prefix not specified - saslServerConfigs.put("plain." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS, + saslServerConfigs.put("plain." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, TestServerCallbackHandler.class); - saslServerConfigs.put("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS, + saslServerConfigs.put("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, DigestServerCallbackHandler.class); server = createEchoServer(securityProtocol); createAndCheckClientConnectionFailure(securityProtocol, "invalid"); // Connections should succeed using the server callback handler associated with the listener ListenerName listener = ListenerName.forSecurityProtocol(securityProtocol); - saslServerConfigs.remove("plain." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS); - saslServerConfigs.remove("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS); - saslServerConfigs.put(listener.saslMechanismConfigPrefix("plain") + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS, + saslServerConfigs.remove("plain." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG); + saslServerConfigs.remove("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG); + saslServerConfigs.put(listener.saslMechanismConfigPrefix("plain") + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, TestServerCallbackHandler.class); - saslServerConfigs.put(listener.saslMechanismConfigPrefix("digest-md5") + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS, + saslServerConfigs.put(listener.saslMechanismConfigPrefix("digest-md5") + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, DigestServerCallbackHandler.class); server.close(); server = createEchoServer(securityProtocol); @@ -1218,7 +1218,7 @@ public void testServerLoginCallbackOverride() throws Exception { jaasConfig.setClientOptions("PLAIN", TestServerCallbackHandler.USERNAME, TestServerCallbackHandler.PASSWORD); ListenerName listenerName = ListenerName.forSecurityProtocol(securityProtocol); String prefix = listenerName.saslMechanismConfigPrefix("PLAIN"); - saslServerConfigs.put(prefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS, + saslServerConfigs.put(prefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, TestServerCallbackHandler.class); Class loginCallback = TestLoginCallbackHandler.class; @@ -1757,7 +1757,7 @@ public void testRepeatedValidSaslPlainOverSsl() throws Exception { * of the data that the client explicitly sent, and then the client will not * recognize that data and will throw an assertion error. */ - saslServerConfigs.put(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS, + saslServerConfigs.put(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, Double.valueOf(1.1 * 1000L / 0.85).longValue()); server = createEchoServer(securityProtocol); @@ -2144,9 +2144,9 @@ private void authenticateUsingSaslPlainAndCheckConnection(String node, boolean e private TestJaasConfig configureMechanisms(String clientMechanism, List serverMechanisms) { saslClientConfigs.put(SaslConfigs.SASL_MECHANISM, clientMechanism); saslServerConfigs.put(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, serverMechanisms); - saslServerConfigs.put(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS, CONNECTIONS_MAX_REAUTH_MS_VALUE); + saslServerConfigs.put(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, CONNECTIONS_MAX_REAUTH_MS_VALUE); if (serverMechanisms.contains("DIGEST-MD5")) { - saslServerConfigs.put("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS, + saslServerConfigs.put("digest-md5." + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, TestDigestLoginModule.DigestServerCallbackHandler.class.getName()); } return TestJaasConfig.createConfiguration(clientMechanism, serverMechanisms); @@ -2154,7 +2154,7 @@ private TestJaasConfig configureMechanisms(String clientMechanism, List private void configureDigestMd5ServerCallback(SecurityProtocol securityProtocol) { String callbackPrefix = ListenerName.forSecurityProtocol(securityProtocol).saslMechanismConfigPrefix("DIGEST-MD5"); - saslServerConfigs.put(callbackPrefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS, + saslServerConfigs.put(callbackPrefix + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, TestDigestLoginModule.DigestServerCallbackHandler.class); } diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index 0c468ba815acf..08e2cebc3124a 100755 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -108,6 +108,7 @@ public void testMurmur2() { @Test public void testGetHost() { + // valid assertEquals("127.0.0.1", getHost("127.0.0.1:8000")); assertEquals("mydomain.com", getHost("PLAINTEXT://mydomain.com:8080")); assertEquals("MyDomain.com", getHost("PLAINTEXT://MyDomain.com:8080")); @@ -116,6 +117,20 @@ public void testGetHost() { assertEquals("2001:db8:85a3:8d3:1319:8a2e:370:7348", getHost("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678")); assertEquals("2001:DB8:85A3:8D3:1319:8A2E:370:7348", getHost("PLAINTEXT://[2001:DB8:85A3:8D3:1319:8A2E:370:7348]:5678")); assertEquals("fe80::b1da:69ca:57f7:63d8%3", getHost("PLAINTEXT://[fe80::b1da:69ca:57f7:63d8%3]:5678")); + + // invalid + assertNull(getHost("PLAINTEXT://mydo)main.com:8080")); + assertNull(getHost("PLAINTEXT://mydo(main.com:8080")); + assertNull(getHost("PLAINTEXT://mydo()main.com:8080")); + assertNull(getHost("PLAINTEXT://mydo(main).com:8080")); + assertNull(getHost("ho)st:9092")); + assertNull(getHost("ho(st:9092")); + assertNull(getHost("ho()st:9092")); + assertNull(getHost("ho(st):9092")); + assertNull(getHost("PLAINTEXT://[2001:db)8:85a3:8d3:1319:8a2e:370:7348]:5678")); + assertNull(getHost("PLAINTEXT://[2001:db(8:85a3:8d3:1319:8a2e:370:7348]:5678")); + assertNull(getHost("PLAINTEXT://[2001:db()8:85a3:8d3:1319:8a2e:370:7348]:5678")); + assertNull(getHost("PLAINTEXT://[2001:db(8:85a3:)8d3:1319:8a2e:370:7348]:5678")); } @Test @@ -130,6 +145,7 @@ public void testHostPattern() { @Test public void testGetPort() { + // valid assertEquals(8000, getPort("127.0.0.1:8000").intValue()); assertEquals(8080, getPort("mydomain.com:8080").intValue()); assertEquals(8080, getPort("MyDomain.com:8080").intValue()); @@ -137,6 +153,12 @@ public void testGetPort() { assertEquals(5678, getPort("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678").intValue()); assertEquals(5678, getPort("[2001:DB8:85A3:8D3:1319:8A2E:370:7348]:5678").intValue()); assertEquals(5678, getPort("[fe80::b1da:69ca:57f7:63d8%3]:5678").intValue()); + + // invalid + assertNull(getPort("host:-92")); + assertNull(getPort("host:-9-2")); + assertNull(getPort("host:92-")); + assertNull(getPort("host:9-2")); } @Test diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java index b883843089955..8115675f5a532 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java @@ -88,7 +88,7 @@ public SchemaBuilder(Type type) { @Override public boolean isOptional() { - return optional == null ? false : optional; + return optional != null && optional; } /** diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java index 7b78c64af0ca7..e144b7c69b6c7 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java @@ -22,25 +22,24 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Serializable; import java.math.BigDecimal; +import java.math.BigInteger; +import java.math.RoundingMode; import java.nio.ByteBuffer; import java.text.CharacterIterator; import java.text.DateFormat; -import java.text.ParseException; +import java.text.ParsePosition; import java.text.SimpleDateFormat; import java.text.StringCharacterIterator; import java.util.ArrayList; -import java.util.Arrays; import java.util.Base64; import java.util.Calendar; -import java.util.Collections; -import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; -import java.util.Set; import java.util.TimeZone; import java.util.regex.Pattern; @@ -61,42 +60,16 @@ */ public class Values { - private static final Logger LOG = LoggerFactory.getLogger(Values.class); - private static final TimeZone UTC = TimeZone.getTimeZone("UTC"); private static final SchemaAndValue NULL_SCHEMA_AND_VALUE = new SchemaAndValue(null, null); - private static final SchemaAndValue TRUE_SCHEMA_AND_VALUE = new SchemaAndValue(Schema.BOOLEAN_SCHEMA, Boolean.TRUE); - private static final SchemaAndValue FALSE_SCHEMA_AND_VALUE = new SchemaAndValue(Schema.BOOLEAN_SCHEMA, Boolean.FALSE); private static final Schema ARRAY_SELECTOR_SCHEMA = SchemaBuilder.array(Schema.STRING_SCHEMA).build(); private static final Schema MAP_SELECTOR_SCHEMA = SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA).build(); private static final Schema STRUCT_SELECTOR_SCHEMA = SchemaBuilder.struct().build(); - private static final String TRUE_LITERAL = Boolean.TRUE.toString(); - private static final String FALSE_LITERAL = Boolean.FALSE.toString(); private static final long MILLIS_PER_DAY = 24 * 60 * 60 * 1000; private static final String NULL_VALUE = "null"; static final String ISO_8601_DATE_FORMAT_PATTERN = "yyyy-MM-dd"; static final String ISO_8601_TIME_FORMAT_PATTERN = "HH:mm:ss.SSS'Z'"; static final String ISO_8601_TIMESTAMP_FORMAT_PATTERN = ISO_8601_DATE_FORMAT_PATTERN + "'T'" + ISO_8601_TIME_FORMAT_PATTERN; - private static final Set TEMPORAL_LOGICAL_TYPE_NAMES = - Collections.unmodifiableSet( - new HashSet<>( - Arrays.asList(Time.LOGICAL_NAME, - Timestamp.LOGICAL_NAME, - Date.LOGICAL_NAME - ) - ) - ); - - private static final String QUOTE_DELIMITER = "\""; - private static final String COMMA_DELIMITER = ","; - private static final String ENTRY_DELIMITER = ":"; - private static final String ARRAY_BEGIN_DELIMITER = "["; - private static final String ARRAY_END_DELIMITER = "]"; - private static final String MAP_BEGIN_DELIMITER = "{"; - private static final String MAP_END_DELIMITER = "}"; - private static final int ISO_8601_DATE_LENGTH = ISO_8601_DATE_FORMAT_PATTERN.length(); - private static final int ISO_8601_TIME_LENGTH = ISO_8601_TIME_FORMAT_PATTERN.length() - 2; // subtract single quotes - private static final int ISO_8601_TIMESTAMP_LENGTH = ISO_8601_TIMESTAMP_FORMAT_PATTERN.length() - 4; // subtract single quotes private static final Pattern TWO_BACKSLASHES = Pattern.compile("\\\\"); @@ -112,7 +85,17 @@ public class Values { * @throws DataException if the value could not be converted to a boolean */ public static Boolean convertToBoolean(Schema schema, Object value) throws DataException { - return (Boolean) convertTo(Schema.OPTIONAL_BOOLEAN_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Boolean) { + return (Boolean) value; + } else if (value instanceof String) { + SchemaAndValue parsed = parseString(value.toString()); + if (parsed.value() instanceof Boolean) { + return (Boolean) parsed.value(); + } + } + return asLong(value, schema, null) == 0L ? Boolean.FALSE : Boolean.TRUE; } /** @@ -125,7 +108,12 @@ public static Boolean convertToBoolean(Schema schema, Object value) throws DataE * @throws DataException if the value could not be converted to a byte */ public static Byte convertToByte(Schema schema, Object value) throws DataException { - return (Byte) convertTo(Schema.OPTIONAL_INT8_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Byte) { + return (Byte) value; + } + return (byte) asLong(value, schema, null); } /** @@ -138,7 +126,12 @@ public static Byte convertToByte(Schema schema, Object value) throws DataExcepti * @throws DataException if the value could not be converted to a short */ public static Short convertToShort(Schema schema, Object value) throws DataException { - return (Short) convertTo(Schema.OPTIONAL_INT16_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Short) { + return (Short) value; + } + return (short) asLong(value, schema, null); } /** @@ -151,7 +144,12 @@ public static Short convertToShort(Schema schema, Object value) throws DataExcep * @throws DataException if the value could not be converted to an integer */ public static Integer convertToInteger(Schema schema, Object value) throws DataException { - return (Integer) convertTo(Schema.OPTIONAL_INT32_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Integer) { + return (Integer) value; + } + return (int) asLong(value, schema, null); } /** @@ -164,7 +162,12 @@ public static Integer convertToInteger(Schema schema, Object value) throws DataE * @throws DataException if the value could not be converted to a long */ public static Long convertToLong(Schema schema, Object value) throws DataException { - return (Long) convertTo(Schema.OPTIONAL_INT64_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Long) { + return (Long) value; + } + return asLong(value, schema, null); } /** @@ -177,7 +180,12 @@ public static Long convertToLong(Schema schema, Object value) throws DataExcepti * @throws DataException if the value could not be converted to a float */ public static Float convertToFloat(Schema schema, Object value) throws DataException { - return (Float) convertTo(Schema.OPTIONAL_FLOAT32_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Float) { + return (Float) value; + } + return (float) asDouble(value, schema, null); } /** @@ -190,7 +198,12 @@ public static Float convertToFloat(Schema schema, Object value) throws DataExcep * @throws DataException if the value could not be converted to a double */ public static Double convertToDouble(Schema schema, Object value) throws DataException { - return (Double) convertTo(Schema.OPTIONAL_FLOAT64_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Double) { + return (Double) value; + } + return asDouble(value, schema, null); } /** @@ -202,7 +215,12 @@ public static Double convertToDouble(Schema schema, Object value) throws DataExc * @return the representation as a string, or null if the supplied value was null */ public static String convertToString(Schema schema, Object value) { - return (String) convertTo(Schema.OPTIONAL_STRING_SCHEMA, schema, value); + if (value == null) { + return null; + } + StringBuilder sb = new StringBuilder(); + append(sb, value, false); + return sb.toString(); } /** @@ -219,7 +237,7 @@ public static String convertToString(Schema schema, Object value) { * @throws DataException if the value cannot be converted to a list value */ public static List convertToList(Schema schema, Object value) { - return (List) convertTo(ARRAY_SELECTOR_SCHEMA, schema, value); + return convertToArray(ARRAY_SELECTOR_SCHEMA, value); } /** @@ -235,7 +253,7 @@ public static List convertToList(Schema schema, Object value) { * @throws DataException if the value cannot be converted to a map value */ public static Map convertToMap(Schema schema, Object value) { - return (Map) convertTo(MAP_SELECTOR_SCHEMA, schema, value); + return convertToMapInternal(MAP_SELECTOR_SCHEMA, value); } /** @@ -250,7 +268,7 @@ public static List convertToList(Schema schema, Object value) { * @throws DataException if the value is not a struct */ public static Struct convertToStruct(Schema schema, Object value) { - return (Struct) convertTo(STRUCT_SELECTOR_SCHEMA, schema, value); + return convertToStructInternal(STRUCT_SELECTOR_SCHEMA, value); } /** @@ -263,7 +281,10 @@ public static Struct convertToStruct(Schema schema, Object value) { * @throws DataException if the value cannot be converted to a time value */ public static java.util.Date convertToTime(Schema schema, Object value) { - return (java.util.Date) convertTo(Time.SCHEMA, schema, value); + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } + return convertToTime(Time.SCHEMA, schema, value); } /** @@ -276,7 +297,10 @@ public static java.util.Date convertToTime(Schema schema, Object value) { * @throws DataException if the value cannot be converted to a date value */ public static java.util.Date convertToDate(Schema schema, Object value) { - return (java.util.Date) convertTo(Date.SCHEMA, schema, value); + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } + return convertToDate(Date.SCHEMA, schema, value); } /** @@ -289,7 +313,10 @@ public static java.util.Date convertToDate(Schema schema, Object value) { * @throws DataException if the value cannot be converted to a timestamp value */ public static java.util.Date convertToTimestamp(Schema schema, Object value) { - return (java.util.Date) convertTo(Timestamp.SCHEMA, schema, value); + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } + return convertToTimestamp(Timestamp.SCHEMA, schema, value); } /** @@ -302,7 +329,10 @@ public static java.util.Date convertToTimestamp(Schema schema, Object value) { * @throws DataException if the value cannot be converted to a decimal value */ public static BigDecimal convertToDecimal(Schema schema, Object value, int scale) { - return (BigDecimal) convertTo(Decimal.schema(scale), schema, value); + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } + return convertToDecimal(Decimal.schema(scale), value); } /** @@ -314,65 +344,59 @@ public static BigDecimal convertToDecimal(Schema schema, Object value, int scale public static Schema inferSchema(Object value) { if (value instanceof String) { return Schema.STRING_SCHEMA; - } - if (value instanceof Boolean) { + } else if (value instanceof Boolean) { return Schema.BOOLEAN_SCHEMA; - } - if (value instanceof Byte) { + } else if (value instanceof Byte) { return Schema.INT8_SCHEMA; - } - if (value instanceof Short) { + } else if (value instanceof Short) { return Schema.INT16_SCHEMA; - } - if (value instanceof Integer) { + } else if (value instanceof Integer) { return Schema.INT32_SCHEMA; - } - if (value instanceof Long) { + } else if (value instanceof Long) { return Schema.INT64_SCHEMA; - } - if (value instanceof Float) { + } else if (value instanceof Float) { return Schema.FLOAT32_SCHEMA; - } - if (value instanceof Double) { + } else if (value instanceof Double) { return Schema.FLOAT64_SCHEMA; - } - if (value instanceof byte[] || value instanceof ByteBuffer) { + } else if (value instanceof byte[] || value instanceof ByteBuffer) { return Schema.BYTES_SCHEMA; + } else if (value instanceof List) { + return inferListSchema((List) value); + } else if (value instanceof Map) { + return inferMapSchema((Map) value); + } else if (value instanceof Struct) { + return ((Struct) value).schema(); } - if (value instanceof List) { - List list = (List) value; - if (list.isEmpty()) { + return null; + } + + private static Schema inferListSchema(List list) { + if (list.isEmpty()) { + return null; + } + SchemaDetector detector = new SchemaDetector(); + for (Object element : list) { + if (!detector.canDetect(element)) { return null; } - SchemaDetector detector = new SchemaDetector(); - for (Object element : list) { - if (!detector.canDetect(element)) { - return null; - } - } - return SchemaBuilder.array(detector.schema()).build(); } - if (value instanceof Map) { - Map map = (Map) value; - if (map.isEmpty()) { + return SchemaBuilder.array(detector.schema()).build(); + } + + private static Schema inferMapSchema(Map map) { + if (map.isEmpty()) { + return null; + } + SchemaDetector keyDetector = new SchemaDetector(); + SchemaDetector valueDetector = new SchemaDetector(); + for (Map.Entry entry : map.entrySet()) { + if (!keyDetector.canDetect(entry.getKey()) || !valueDetector.canDetect(entry.getValue())) { return null; } - SchemaDetector keyDetector = new SchemaDetector(); - SchemaDetector valueDetector = new SchemaDetector(); - for (Map.Entry entry : map.entrySet()) { - if (!keyDetector.canDetect(entry.getKey()) || !valueDetector.canDetect(entry.getValue())) { - return null; - } - } - return SchemaBuilder.map(keyDetector.schema(), valueDetector.schema()).build(); } - if (value instanceof Struct) { - return ((Struct) value).schema(); - } - return null; + return SchemaBuilder.map(keyDetector.schema(), valueDetector.schema()).build(); } - /** * Parse the specified string representation of a value into its schema and value. * @@ -387,8 +411,8 @@ public static SchemaAndValue parseString(String value) { if (value.isEmpty()) { return new SchemaAndValue(Schema.STRING_SCHEMA, value); } - Parser parser = new Parser(value); - return parse(parser, false); + ValueParser parser = new ValueParser(new Parser(value)); + return parser.parse(false); } /** @@ -396,7 +420,7 @@ public static SchemaAndValue parseString(String value) { * * @param toSchema the schema for the desired type; may not be null * @param fromSchema the schema for the supplied value; may be null if not known - * @return the converted value; never null + * @return the converted value; null if the passed-in schema was optional, and the input value was null. * @throws DataException if the value could not be converted to the desired type */ protected static Object convertTo(Schema toSchema, Schema fromSchema, Object value) throws DataException { @@ -408,184 +432,213 @@ protected static Object convertTo(Schema toSchema, Schema fromSchema, Object val } switch (toSchema.type()) { case BYTES: - if (Decimal.LOGICAL_NAME.equals(toSchema.name())) { - if (value instanceof ByteBuffer) { - value = Utils.toArray((ByteBuffer) value); - } - if (value instanceof byte[]) { - return Decimal.toLogical(toSchema, (byte[]) value); - } - if (value instanceof BigDecimal) { - return value; - } - if (value instanceof Number) { - // Not already a decimal, so treat it as a double ... - double converted = ((Number) value).doubleValue(); - return BigDecimal.valueOf(converted); - } - if (value instanceof String) { - return new BigDecimal(value.toString()); - } - } - if (value instanceof ByteBuffer) { - return Utils.toArray((ByteBuffer) value); - } - if (value instanceof byte[]) { - return value; - } - if (value instanceof BigDecimal) { - return Decimal.fromLogical(toSchema, (BigDecimal) value); - } - break; + return convertMaybeLogicalBytes(toSchema, value); case STRING: - StringBuilder sb = new StringBuilder(); - append(sb, value, false); - return sb.toString(); + return convertToString(fromSchema, value); case BOOLEAN: - if (value instanceof Boolean) { - return value; - } - if (value instanceof String) { - SchemaAndValue parsed = parseString(value.toString()); - if (parsed.value() instanceof Boolean) { - return parsed.value(); - } - } - return asLong(value, fromSchema, null) == 0L ? Boolean.FALSE : Boolean.TRUE; + return convertToBoolean(fromSchema, value); case INT8: - if (value instanceof Byte) { - return value; - } - return (byte) asLong(value, fromSchema, null); + return convertToByte(fromSchema, value); case INT16: - if (value instanceof Short) { - return value; - } - return (short) asLong(value, fromSchema, null); + return convertToShort(fromSchema, value); case INT32: - if (Date.LOGICAL_NAME.equals(toSchema.name())) { - if (value instanceof String) { - SchemaAndValue parsed = parseString(value.toString()); - value = parsed.value(); - } - if (value instanceof java.util.Date) { - if (fromSchema != null) { - String fromSchemaName = fromSchema.name(); - if (Date.LOGICAL_NAME.equals(fromSchemaName)) { - return value; - } - if (Timestamp.LOGICAL_NAME.equals(fromSchemaName)) { - // Just get the number of days from this timestamp - long millis = ((java.util.Date) value).getTime(); - int days = (int) (millis / MILLIS_PER_DAY); // truncates - return Date.toLogical(toSchema, days); - } - } else { - // There is no fromSchema, so no conversion is needed - return value; - } - } - long numeric = asLong(value, fromSchema, null); - return Date.toLogical(toSchema, (int) numeric); - } - if (Time.LOGICAL_NAME.equals(toSchema.name())) { - if (value instanceof String) { - SchemaAndValue parsed = parseString(value.toString()); - value = parsed.value(); - } - if (value instanceof java.util.Date) { - if (fromSchema != null) { - String fromSchemaName = fromSchema.name(); - if (Time.LOGICAL_NAME.equals(fromSchemaName)) { - return value; - } - if (Timestamp.LOGICAL_NAME.equals(fromSchemaName)) { - // Just get the time portion of this timestamp - Calendar calendar = Calendar.getInstance(UTC); - calendar.setTime((java.util.Date) value); - calendar.set(Calendar.YEAR, 1970); - calendar.set(Calendar.MONTH, 0); // Months are zero-based - calendar.set(Calendar.DAY_OF_MONTH, 1); - return Time.toLogical(toSchema, (int) calendar.getTimeInMillis()); - } - } else { - // There is no fromSchema, so no conversion is needed - return value; - } - } - long numeric = asLong(value, fromSchema, null); - return Time.toLogical(toSchema, (int) numeric); - } - if (value instanceof Integer) { - return value; - } - return (int) asLong(value, fromSchema, null); + return convertMaybeLogicalInteger(toSchema, fromSchema, value); case INT64: - if (Timestamp.LOGICAL_NAME.equals(toSchema.name())) { - if (value instanceof String) { - SchemaAndValue parsed = parseString(value.toString()); - value = parsed.value(); - } - if (value instanceof java.util.Date) { - java.util.Date date = (java.util.Date) value; - if (fromSchema != null) { - String fromSchemaName = fromSchema.name(); - if (Date.LOGICAL_NAME.equals(fromSchemaName)) { - int days = Date.fromLogical(fromSchema, date); - long millis = days * MILLIS_PER_DAY; - return Timestamp.toLogical(toSchema, millis); - } - if (Time.LOGICAL_NAME.equals(fromSchemaName)) { - long millis = Time.fromLogical(fromSchema, date); - return Timestamp.toLogical(toSchema, millis); - } - if (Timestamp.LOGICAL_NAME.equals(fromSchemaName)) { - return value; - } - } else { - // There is no fromSchema, so no conversion is needed - return value; - } - } - long numeric = asLong(value, fromSchema, null); - return Timestamp.toLogical(toSchema, numeric); - } - if (value instanceof Long) { - return value; - } - return asLong(value, fromSchema, null); + return convertMaybeLogicalLong(toSchema, fromSchema, value); case FLOAT32: - if (value instanceof Float) { - return value; - } - return (float) asDouble(value, fromSchema, null); + return convertToFloat(fromSchema, value); case FLOAT64: - if (value instanceof Double) { - return value; - } - return asDouble(value, fromSchema, null); + return convertToDouble(fromSchema, value); case ARRAY: - if (value instanceof String) { - SchemaAndValue schemaAndValue = parseString(value.toString()); - value = schemaAndValue.value(); + return convertToArray(toSchema, value); + case MAP: + return convertToMapInternal(toSchema, value); + case STRUCT: + return convertToStructInternal(toSchema, value); + } + throw new DataException("Unable to convert " + value + " (" + value.getClass() + ") to " + toSchema); + } + + private static Serializable convertMaybeLogicalBytes(Schema toSchema, Object value) { + if (Decimal.LOGICAL_NAME.equals(toSchema.name())) { + return convertToDecimal(toSchema, value); + } + return convertToBytes(toSchema, value); + } + + private static BigDecimal convertToDecimal(Schema toSchema, Object value) { + if (value instanceof ByteBuffer) { + value = Utils.toArray((ByteBuffer) value); + } + if (value instanceof byte[]) { + return Decimal.toLogical(toSchema, (byte[]) value); + } + if (value instanceof BigDecimal) { + return (BigDecimal) value; + } + if (value instanceof Number) { + // Not already a decimal, so treat it as a double ... + double converted = ((Number) value).doubleValue(); + return BigDecimal.valueOf(converted); + } + if (value instanceof String) { + return new BigDecimal(value.toString()); + } + throw new DataException("Unable to convert " + value + " (" + value.getClass() + ") to " + toSchema); + } + + private static byte[] convertToBytes(Schema toSchema, Object value) { + if (value instanceof ByteBuffer) { + return Utils.toArray((ByteBuffer) value); + } + if (value instanceof byte[]) { + return (byte[]) value; + } + if (value instanceof BigDecimal) { + return Decimal.fromLogical(toSchema, (BigDecimal) value); + } + throw new DataException("Unable to convert " + value + " (" + value.getClass() + ") to " + toSchema); + } + + private static Serializable convertMaybeLogicalInteger(Schema toSchema, Schema fromSchema, Object value) { + if (Date.LOGICAL_NAME.equals(toSchema.name())) { + return convertToDate(toSchema, fromSchema, value); + } + if (Time.LOGICAL_NAME.equals(toSchema.name())) { + return convertToTime(toSchema, fromSchema, value); + } + return convertToInteger(fromSchema, value); + } + + private static java.util.Date convertToDate(Schema toSchema, Schema fromSchema, Object value) { + if (value == null) { + return null; + } else if (value instanceof String) { + SchemaAndValue parsed = parseString(value.toString()); + value = parsed.value(); + } + if (value instanceof java.util.Date) { + if (fromSchema != null) { + String fromSchemaName = fromSchema.name(); + if (Date.LOGICAL_NAME.equals(fromSchemaName)) { + return (java.util.Date) value; } - if (value instanceof List) { - return value; + if (Timestamp.LOGICAL_NAME.equals(fromSchemaName)) { + // Just get the number of days from this timestamp + long millis = ((java.util.Date) value).getTime(); + int days = (int) (millis / MILLIS_PER_DAY); // truncates + return Date.toLogical(toSchema, days); } - break; - case MAP: - if (value instanceof String) { - SchemaAndValue schemaAndValue = parseString(value.toString()); - value = schemaAndValue.value(); + } else { + // There is no fromSchema, so no conversion is needed + return (java.util.Date) value; + } + } + long numeric = asLong(value, fromSchema, null); + return Date.toLogical(toSchema, (int) numeric); + } + + private static java.util.Date convertToTime(Schema toSchema, Schema fromSchema, Object value) { + if (value == null) { + return null; + } else if (value instanceof String) { + SchemaAndValue parsed = parseString(value.toString()); + value = parsed.value(); + } + if (value instanceof java.util.Date) { + if (fromSchema != null) { + String fromSchemaName = fromSchema.name(); + if (Time.LOGICAL_NAME.equals(fromSchemaName)) { + return (java.util.Date) value; } - if (value instanceof Map) { - return value; + if (Timestamp.LOGICAL_NAME.equals(fromSchemaName)) { + // Just get the time portion of this timestamp + Calendar calendar = Calendar.getInstance(UTC); + calendar.setTime((java.util.Date) value); + calendar.set(Calendar.YEAR, 1970); + calendar.set(Calendar.MONTH, 0); // Months are zero-based + calendar.set(Calendar.DAY_OF_MONTH, 1); + return Time.toLogical(toSchema, (int) calendar.getTimeInMillis()); } - break; - case STRUCT: - if (value instanceof Struct) { - return value; + } else { + // There is no fromSchema, so no conversion is needed + return (java.util.Date) value; + } + } + long numeric = asLong(value, fromSchema, null); + return Time.toLogical(toSchema, (int) numeric); + } + + private static Serializable convertMaybeLogicalLong(Schema toSchema, Schema fromSchema, Object value) { + if (Timestamp.LOGICAL_NAME.equals(toSchema.name())) { + return convertToTimestamp(toSchema, fromSchema, value); + } + return convertToLong(fromSchema, value); + } + + private static java.util.Date convertToTimestamp(Schema toSchema, Schema fromSchema, Object value) { + if (value == null) { + return null; + } else if (value instanceof String) { + SchemaAndValue parsed = parseString(value.toString()); + value = parsed.value(); + } + if (value instanceof java.util.Date) { + java.util.Date date = (java.util.Date) value; + if (fromSchema != null) { + String fromSchemaName = fromSchema.name(); + if (Date.LOGICAL_NAME.equals(fromSchemaName)) { + int days = Date.fromLogical(fromSchema, date); + long millis = days * MILLIS_PER_DAY; + return Timestamp.toLogical(toSchema, millis); } + if (Time.LOGICAL_NAME.equals(fromSchemaName)) { + long millis = Time.fromLogical(fromSchema, date); + return Timestamp.toLogical(toSchema, millis); + } + if (Timestamp.LOGICAL_NAME.equals(fromSchemaName)) { + return date; + } + } else { + // There is no fromSchema, so no conversion is needed + return date; + } + } + long numeric = asLong(value, fromSchema, null); + return Timestamp.toLogical(toSchema, numeric); + } + + private static List convertToArray(Schema toSchema, Object value) { + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } else if (value instanceof String) { + SchemaAndValue schemaAndValue = parseString(value.toString()); + value = schemaAndValue.value(); + } + if (value instanceof List) { + return (List) value; + } + throw new DataException("Unable to convert " + value + " (" + value.getClass() + ") to " + toSchema); + } + + private static Map convertToMapInternal(Schema toSchema, Object value) { + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } else if (value instanceof String) { + SchemaAndValue schemaAndValue = parseString(value.toString()); + value = schemaAndValue.value(); + } + if (value instanceof Map) { + return (Map) value; + } + throw new DataException("Unable to convert " + value + " (" + value.getClass() + ") to " + toSchema); + } + + private static Struct convertToStructInternal(Schema toSchema, Object value) { + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } else if (value instanceof Struct) { + return (Struct) value; } throw new DataException("Unable to convert " + value + " (" + value.getClass() + ") to " + toSchema); } @@ -744,174 +797,212 @@ public static DateFormat dateFormatFor(java.util.Date value) { return new SimpleDateFormat(ISO_8601_TIMESTAMP_FORMAT_PATTERN); } - protected static boolean canParseSingleTokenLiteral(Parser parser, boolean embedded, String tokenLiteral) { - int startPosition = parser.mark(); - // If the next token is what we expect, then either... - if (parser.canConsume(tokenLiteral)) { - // ...we're reading an embedded value, in which case the next token will be handled appropriately - // by the caller if it's something like an end delimiter for a map or array, or a comma to - // separate multiple embedded values... - // ...or it's being parsed as part of a top-level string, in which case, any other tokens should - // cause use to stop parsing this single-token literal as such and instead just treat it like - // a string. For example, the top-level string "true}" will be tokenized as the tokens "true" and - // "}", but should ultimately be parsed as just the string "true}" instead of the boolean true. - if (embedded || !parser.hasNext()) { - return true; - } + private static class ValueParser { + + private static final Logger log = LoggerFactory.getLogger(ValueParser.class); + private static final SchemaAndValue TRUE_SCHEMA_AND_VALUE = new SchemaAndValue(Schema.BOOLEAN_SCHEMA, Boolean.TRUE); + private static final SchemaAndValue FALSE_SCHEMA_AND_VALUE = new SchemaAndValue(Schema.BOOLEAN_SCHEMA, Boolean.FALSE); + private static final String TRUE_LITERAL = Boolean.TRUE.toString(); + private static final String FALSE_LITERAL = Boolean.FALSE.toString(); + private static final BigInteger LONG_MIN = BigInteger.valueOf(Long.MIN_VALUE); + private static final BigInteger LONG_MAX = BigInteger.valueOf(Long.MAX_VALUE); + private static final String QUOTE_DELIMITER = "\""; + private static final String COMMA_DELIMITER = ","; + private static final String ENTRY_DELIMITER = ":"; + private static final String ARRAY_BEGIN_DELIMITER = "["; + private static final String ARRAY_END_DELIMITER = "]"; + private static final String MAP_BEGIN_DELIMITER = "{"; + private static final String MAP_END_DELIMITER = "}"; + private static final int ISO_8601_DATE_LENGTH = ISO_8601_DATE_FORMAT_PATTERN.length(); + private static final int ISO_8601_TIME_LENGTH = ISO_8601_TIME_FORMAT_PATTERN.length() - 2; // subtract single quotes + private static final int ISO_8601_TIMESTAMP_LENGTH = ISO_8601_TIMESTAMP_FORMAT_PATTERN.length() - 4; // subtract single quotes + + private final Parser parser; + + private ValueParser(Parser parser) { + this.parser = parser; } - parser.rewindTo(startPosition); - return false; - } - protected static SchemaAndValue parse(Parser parser, boolean embedded) throws NoSuchElementException { - if (!parser.hasNext()) { - return null; - } - if (embedded) { - if (parser.canConsume(QUOTE_DELIMITER)) { - StringBuilder sb = new StringBuilder(); - while (parser.hasNext()) { - if (parser.canConsume(QUOTE_DELIMITER)) { - break; - } - sb.append(parser.next()); - } - String content = sb.toString(); - // We can parse string literals as temporal logical types, but all others - // are treated as strings - SchemaAndValue parsed = parseString(content); - if (parsed != null && TEMPORAL_LOGICAL_TYPE_NAMES.contains(parsed.schema().name())) { - return parsed; + private boolean canParseSingleTokenLiteral(boolean embedded, String tokenLiteral) { + int startPosition = parser.mark(); + // If the next token is what we expect, then either... + if (parser.canConsume(tokenLiteral)) { + // ...we're reading an embedded value, in which case the next token will be handled appropriately + // by the caller if it's something like an end delimiter for a map or array, or a comma to + // separate multiple embedded values... + // ...or it's being parsed as part of a top-level string, in which case, any other tokens should + // cause use to stop parsing this single-token literal as such and instead just treat it like + // a string. For example, the top-level string "true}" will be tokenized as the tokens "true" and + // "}", but should ultimately be parsed as just the string "true}" instead of the boolean true. + if (embedded || !parser.hasNext()) { + return true; } - return new SchemaAndValue(Schema.STRING_SCHEMA, content); } + parser.rewindTo(startPosition); + return false; } - if (canParseSingleTokenLiteral(parser, embedded, NULL_VALUE)) { - return null; - } - if (canParseSingleTokenLiteral(parser, embedded, TRUE_LITERAL)) { - return TRUE_SCHEMA_AND_VALUE; - } - if (canParseSingleTokenLiteral(parser, embedded, FALSE_LITERAL)) { - return FALSE_SCHEMA_AND_VALUE; + public SchemaAndValue parse(boolean embedded) throws NoSuchElementException { + if (!parser.hasNext()) { + return null; + } else if (embedded && parser.canConsume(QUOTE_DELIMITER)) { + return parseQuotedString(); + } else if (canParseSingleTokenLiteral(embedded, NULL_VALUE)) { + return null; + } else if (canParseSingleTokenLiteral(embedded, TRUE_LITERAL)) { + return TRUE_SCHEMA_AND_VALUE; + } else if (canParseSingleTokenLiteral(embedded, FALSE_LITERAL)) { + return FALSE_SCHEMA_AND_VALUE; + } + + int startPosition = parser.mark(); + + try { + if (parser.canConsume(ARRAY_BEGIN_DELIMITER)) { + return parseArray(); + } else if (parser.canConsume(MAP_BEGIN_DELIMITER)) { + return parseMap(); + } + } catch (DataException e) { + log.trace("Unable to parse the value as a map or an array; reverting to string", e); + parser.rewindTo(startPosition); + } + + String token = parser.next(); + if (Utils.isBlank(token)) { + return new SchemaAndValue(Schema.STRING_SCHEMA, token); + } else { + return parseNextToken(embedded, token.trim()); + } } - int startPosition = parser.mark(); + private SchemaAndValue parseNextToken(boolean embedded, String token) { + char firstChar = token.charAt(0); + boolean firstCharIsDigit = Character.isDigit(firstChar); - try { - if (parser.canConsume(ARRAY_BEGIN_DELIMITER)) { - List result = new ArrayList<>(); - boolean compatible = true; - Schema elementSchema = null; - while (parser.hasNext()) { - if (parser.canConsume(ARRAY_END_DELIMITER)) { - Schema listSchema; - if (elementSchema != null && compatible) { - listSchema = SchemaBuilder.array(elementSchema).schema(); - result = alignListEntriesWithSchema(listSchema, result); - } else { - // Every value is null - listSchema = SchemaBuilder.arrayOfNull().build(); - } - return new SchemaAndValue(listSchema, result); - } + // Temporal types are more restrictive, so try them first + if (firstCharIsDigit) { + SchemaAndValue temporal = parseMultipleTokensAsTemporal(token); + if (temporal != null) { + return temporal; + } + } + if (firstCharIsDigit || firstChar == '+' || firstChar == '-') { + try { + return parseAsNumber(token); + } catch (NumberFormatException e) { + // can't parse as a number + } + } + if (embedded) { + throw new DataException("Failed to parse embedded value"); + } + // At this point, the only thing this non-embedded value can be is a string. + return new SchemaAndValue(Schema.STRING_SCHEMA, parser.original()); + } - if (parser.canConsume(COMMA_DELIMITER)) { - throw new DataException("Unable to parse an empty array element: " + parser.original()); - } - SchemaAndValue element = parse(parser, true); - elementSchema = commonSchemaFor(elementSchema, element); - if (elementSchema == null && element != null && element.schema() != null) { - compatible = false; - } - result.add(element != null ? element.value() : null); + private SchemaAndValue parseQuotedString() { + StringBuilder sb = new StringBuilder(); + while (parser.hasNext()) { + if (parser.canConsume(QUOTE_DELIMITER)) { + break; + } + sb.append(parser.next()); + } + String content = sb.toString(); + // We can parse string literals as temporal logical types, but all others + // are treated as strings + SchemaAndValue parsed = parseAsTemporal(content); + if (parsed != null) { + return parsed; + } + return new SchemaAndValue(Schema.STRING_SCHEMA, content); + } - int currentPosition = parser.mark(); - if (parser.canConsume(ARRAY_END_DELIMITER)) { - parser.rewindTo(currentPosition); - } else if (!parser.canConsume(COMMA_DELIMITER)) { - throw new DataException("Array elements missing '" + COMMA_DELIMITER + "' delimiter"); + private SchemaAndValue parseArray() { + List result = new ArrayList<>(); + SchemaMerger elementSchema = new SchemaMerger(); + while (parser.hasNext()) { + if (parser.canConsume(ARRAY_END_DELIMITER)) { + Schema listSchema; + if (elementSchema.hasCommonSchema()) { + listSchema = SchemaBuilder.array(elementSchema.schema()).schema(); + result = alignListEntriesWithSchema(listSchema, result); + } else { + // Every value is null + listSchema = SchemaBuilder.arrayOfNull().build(); } + return new SchemaAndValue(listSchema, result); } - // Missing either a comma or an end delimiter - if (COMMA_DELIMITER.equals(parser.previous())) { - throw new DataException("Array is missing element after ',': " + parser.original()); + if (parser.canConsume(COMMA_DELIMITER)) { + throw new DataException("Unable to parse an empty array element: " + parser.original()); + } + SchemaAndValue element = parse(true); + elementSchema.merge(element); + result.add(element != null ? element.value() : null); + + int currentPosition = parser.mark(); + if (parser.canConsume(ARRAY_END_DELIMITER)) { + parser.rewindTo(currentPosition); + } else if (!parser.canConsume(COMMA_DELIMITER)) { + throw new DataException("Array elements missing '" + COMMA_DELIMITER + "' delimiter"); } - throw new DataException("Array is missing terminating ']': " + parser.original()); } - if (parser.canConsume(MAP_BEGIN_DELIMITER)) { - Map result = new LinkedHashMap<>(); - boolean keyCompatible = true; - Schema keySchema = null; - boolean valueCompatible = true; - Schema valueSchema = null; - while (parser.hasNext()) { - if (parser.canConsume(MAP_END_DELIMITER)) { - Schema mapSchema; - if (keySchema != null && valueSchema != null && keyCompatible && valueCompatible) { - mapSchema = SchemaBuilder.map(keySchema, valueSchema).build(); - result = alignMapKeysAndValuesWithSchema(mapSchema, result); - } else if (keySchema != null && keyCompatible) { - mapSchema = SchemaBuilder.mapWithNullValues(keySchema); - result = alignMapKeysWithSchema(mapSchema, result); - } else { - mapSchema = SchemaBuilder.mapOfNull().build(); - } - return new SchemaAndValue(mapSchema, result); - } + // Missing either a comma or an end delimiter + if (COMMA_DELIMITER.equals(parser.previous())) { + throw new DataException("Array is missing element after ',': " + parser.original()); + } + throw new DataException("Array is missing terminating ']': " + parser.original()); + } - if (parser.canConsume(COMMA_DELIMITER)) { - throw new DataException("Unable to parse a map entry with no key or value: " + parser.original()); - } - SchemaAndValue key = parse(parser, true); - if (key == null || key.value() == null) { - throw new DataException("Map entry may not have a null key: " + parser.original()); + private SchemaAndValue parseMap() { + Map result = new LinkedHashMap<>(); + SchemaMerger keySchema = new SchemaMerger(); + SchemaMerger valueSchema = new SchemaMerger(); + while (parser.hasNext()) { + if (parser.canConsume(MAP_END_DELIMITER)) { + Schema mapSchema; + if (keySchema.hasCommonSchema() && valueSchema.hasCommonSchema()) { + mapSchema = SchemaBuilder.map(keySchema.schema(), valueSchema.schema()).build(); + result = alignMapKeysAndValuesWithSchema(mapSchema, result); + } else if (keySchema.hasCommonSchema()) { + mapSchema = SchemaBuilder.mapWithNullValues(keySchema.schema()); + result = alignMapKeysWithSchema(mapSchema, result); + } else { + mapSchema = SchemaBuilder.mapOfNull().build(); } + return new SchemaAndValue(mapSchema, result); + } - if (!parser.canConsume(ENTRY_DELIMITER)) { - throw new DataException("Map entry is missing '" + ENTRY_DELIMITER - + "' at " + parser.position() - + " in " + parser.original()); - } - SchemaAndValue value = parse(parser, true); - Object entryValue = value != null ? value.value() : null; - result.put(key.value(), entryValue); - - parser.canConsume(COMMA_DELIMITER); - keySchema = commonSchemaFor(keySchema, key); - if (keySchema == null && key.schema() != null) { - keyCompatible = false; - } - valueSchema = commonSchemaFor(valueSchema, value); - if (valueSchema == null && value != null && value.schema() != null) { - valueCompatible = false; - } + if (parser.canConsume(COMMA_DELIMITER)) { + throw new DataException("Unable to parse a map entry with no key or value: " + parser.original()); } - // Missing either a comma or an end delimiter - if (COMMA_DELIMITER.equals(parser.previous())) { - throw new DataException("Map is missing element after ',': " + parser.original()); + SchemaAndValue key = parse(true); + if (key == null || key.value() == null) { + throw new DataException("Map entry may not have a null key: " + parser.original()); + } else if (!parser.canConsume(ENTRY_DELIMITER)) { + throw new DataException("Map entry is missing '" + ENTRY_DELIMITER + + "' at " + parser.position() + + " in " + parser.original()); } - throw new DataException("Map is missing terminating '}': " + parser.original()); - } - } catch (DataException e) { - LOG.trace("Unable to parse the value as a map or an array; reverting to string", e); - parser.rewindTo(startPosition); - } + SchemaAndValue value = parse(true); + Object entryValue = value != null ? value.value() : null; + result.put(key.value(), entryValue); - String token = parser.next(); - if (Utils.isBlank(token)) { - return new SchemaAndValue(Schema.STRING_SCHEMA, token); + parser.canConsume(COMMA_DELIMITER); + keySchema.merge(key); + valueSchema.merge(value); + } + // Missing either a comma or an end delimiter + if (COMMA_DELIMITER.equals(parser.previous())) { + throw new DataException("Map is missing element after ',': " + parser.original()); + } + throw new DataException("Map is missing terminating '}': " + parser.original()); } - token = token.trim(); - char firstChar = token.charAt(0); - boolean firstCharIsDigit = Character.isDigit(firstChar); - - // Temporal types are more restrictive, so try them first - if (firstCharIsDigit) { + private SchemaAndValue parseMultipleTokensAsTemporal(String token) { // The time and timestamp literals may be split into 5 tokens since an unescaped colon // is a delimiter. Check these first since the first of these tokens is a simple numeric int position = parser.mark(); @@ -925,143 +1016,146 @@ protected static SchemaAndValue parse(Parser parser, boolean embedded) throws No } // No match was found using the 5 tokens, so rewind and see if the current token has a date, time, or timestamp parser.rewindTo(position); - SchemaAndValue temporal = parseAsTemporal(token); - if (temporal != null) { - return temporal; - } + return parseAsTemporal(token); } - if (firstCharIsDigit || firstChar == '+' || firstChar == '-') { - try { - // Try to parse as a number ... - BigDecimal decimal = new BigDecimal(token); - try { - return new SchemaAndValue(Schema.INT8_SCHEMA, decimal.byteValueExact()); - } catch (ArithmeticException e) { - // continue - } - try { - return new SchemaAndValue(Schema.INT16_SCHEMA, decimal.shortValueExact()); - } catch (ArithmeticException e) { - // continue - } - try { - return new SchemaAndValue(Schema.INT32_SCHEMA, decimal.intValueExact()); - } catch (ArithmeticException e) { - // continue - } - try { - return new SchemaAndValue(Schema.INT64_SCHEMA, decimal.longValueExact()); - } catch (ArithmeticException e) { - // continue - } - float fValue = decimal.floatValue(); - if (fValue != Float.NEGATIVE_INFINITY && fValue != Float.POSITIVE_INFINITY + + private static SchemaAndValue parseAsNumber(String token) { + // Try to parse as a number ... + BigDecimal decimal = new BigDecimal(token); + SchemaAndValue exactDecimal = parseAsExactDecimal(decimal); + float fValue = decimal.floatValue(); + double dValue = decimal.doubleValue(); + if (exactDecimal != null) { + return exactDecimal; + } else if (fValue != Float.NEGATIVE_INFINITY && fValue != Float.POSITIVE_INFINITY && decimal.scale() != 0) { - return new SchemaAndValue(Schema.FLOAT32_SCHEMA, fValue); - } - double dValue = decimal.doubleValue(); - if (dValue != Double.NEGATIVE_INFINITY && dValue != Double.POSITIVE_INFINITY + return new SchemaAndValue(Schema.FLOAT32_SCHEMA, fValue); + } else if (dValue != Double.NEGATIVE_INFINITY && dValue != Double.POSITIVE_INFINITY && decimal.scale() != 0) { - return new SchemaAndValue(Schema.FLOAT64_SCHEMA, dValue); - } + return new SchemaAndValue(Schema.FLOAT64_SCHEMA, dValue); + } else { Schema schema = Decimal.schema(decimal.scale()); return new SchemaAndValue(schema, decimal); - } catch (NumberFormatException e) { - // can't parse as a number } } - if (embedded) { - throw new DataException("Failed to parse embedded value"); - } - // At this point, the only thing this non-embedded value can be is a string. - return new SchemaAndValue(Schema.STRING_SCHEMA, parser.original()); - } - private static SchemaAndValue parseAsTemporal(String token) { - if (token == null) { + private static SchemaAndValue parseAsExactDecimal(BigDecimal decimal) { + BigDecimal ceil = decimal.setScale(0, RoundingMode.CEILING); + BigDecimal floor = decimal.setScale(0, RoundingMode.FLOOR); + if (ceil.equals(floor)) { + BigInteger num = ceil.toBigIntegerExact(); + if (ceil.precision() >= 19 && (num.compareTo(LONG_MIN) < 0 || num.compareTo(LONG_MAX) > 0)) { + return null; + } + long integral = num.longValue(); + byte int8 = (byte) integral; + short int16 = (short) integral; + int int32 = (int) integral; + if (int8 == integral) { + return new SchemaAndValue(Schema.INT8_SCHEMA, int8); + } else if (int16 == integral) { + return new SchemaAndValue(Schema.INT16_SCHEMA, int16); + } else if (int32 == integral) { + return new SchemaAndValue(Schema.INT32_SCHEMA, int32); + } else { + return new SchemaAndValue(Schema.INT64_SCHEMA, integral); + } + } return null; } - // If the colons were escaped, we'll see the escape chars and need to remove them - token = token.replace("\\:", ":"); - int tokenLength = token.length(); - if (tokenLength == ISO_8601_TIME_LENGTH) { - try { - return new SchemaAndValue(Time.SCHEMA, new SimpleDateFormat(ISO_8601_TIME_FORMAT_PATTERN).parse(token)); - } catch (ParseException e) { - // not a valid date + + private static SchemaAndValue parseAsTemporal(String token) { + if (token == null) { + return null; } - } else if (tokenLength == ISO_8601_TIMESTAMP_LENGTH) { - try { - return new SchemaAndValue(Timestamp.SCHEMA, new SimpleDateFormat(ISO_8601_TIMESTAMP_FORMAT_PATTERN).parse(token)); - } catch (ParseException e) { - // not a valid date + // If the colons were escaped, we'll see the escape chars and need to remove them + token = token.replace("\\:", ":"); + int tokenLength = token.length(); + if (tokenLength == ISO_8601_TIME_LENGTH) { + return parseAsTemporalType(token, Time.SCHEMA, ISO_8601_TIME_FORMAT_PATTERN); + } else if (tokenLength == ISO_8601_TIMESTAMP_LENGTH) { + return parseAsTemporalType(token, Timestamp.SCHEMA, ISO_8601_TIMESTAMP_FORMAT_PATTERN); + } else if (tokenLength == ISO_8601_DATE_LENGTH) { + return parseAsTemporalType(token, Date.SCHEMA, ISO_8601_DATE_FORMAT_PATTERN); + } else { + return null; } - } else if (tokenLength == ISO_8601_DATE_LENGTH) { - try { - return new SchemaAndValue(Date.SCHEMA, new SimpleDateFormat(ISO_8601_DATE_FORMAT_PATTERN).parse(token)); - } catch (ParseException e) { - // not a valid date + } + + private static SchemaAndValue parseAsTemporalType(String token, Schema schema, String pattern) { + ParsePosition pos = new ParsePosition(0); + java.util.Date result = new SimpleDateFormat(pattern).parse(token, pos); + if (pos.getIndex() != 0) { + return new SchemaAndValue(schema, result); } + return null; } - return null; } - protected static Schema commonSchemaFor(Schema previous, SchemaAndValue latest) { - if (latest == null) { - return previous; + /** + * Utility for merging various optional primitive numeric schemas into a common schema. + * If a non-numeric type appears (including logical numeric types), no common schema will be inferred. + * This class is not thread-safe and should only be accessed by one thread. + */ + private static class SchemaMerger { + /** + * Schema which applies to all of the values passed to {@link #merge(SchemaAndValue)} + * Null if no non-null schemas have been seen, or if the values seen do not have a common schema + */ + private Schema common = null; + /** + * Flag to determine the meaning of the null sentinel in {@link #common} + * If true, null means "any optional type", as no non-null values have appeared. + * If false, null means "no common type", as one or more non-null values had mutually exclusive schemas. + */ + private boolean compatible = true; + + protected void merge(SchemaAndValue latest) { + if (latest != null && latest.schema() != null && compatible) { + if (common == null) { + // This null means any type is valid, so choose the new schema. + common = latest.schema(); + } else { + // There is a previous type restriction, so merge the new schema into the old one. + common = mergeSchemas(common, latest.schema()); + // If there isn't a common schema any longer, then give up on finding further compatible schemas. + compatible = common != null; + } + } } - if (previous == null) { - return latest.schema(); + + protected boolean hasCommonSchema() { + return common != null; } - Schema newSchema = latest.schema(); + + protected Schema schema() { + return common; + } + } + + /** + * Merge two schemas to a common schema which can represent values from both input schemas. + * @param previous One Schema, non-null + * @param newSchema Another schema, non-null + * @return A schema that is a superset of both input schemas, or null if no common schema is found. + */ + private static Schema mergeSchemas(Schema previous, Schema newSchema) { Type previousType = previous.type(); Type newType = newSchema.type(); if (previousType != newType) { switch (previous.type()) { case INT8: - if (newType == Type.INT16 || newType == Type.INT32 || newType == Type.INT64 || newType == Type.FLOAT32 || newType == - Type.FLOAT64) { - return newSchema; - } - break; + return commonSchemaForInt8(newSchema, newType); case INT16: - if (newType == Type.INT8) { - return previous; - } - if (newType == Type.INT32 || newType == Type.INT64 || newType == Type.FLOAT32 || newType == Type.FLOAT64) { - return newSchema; - } - break; + return commonSchemaForInt16(previous, newSchema, newType); case INT32: - if (newType == Type.INT8 || newType == Type.INT16) { - return previous; - } - if (newType == Type.INT64 || newType == Type.FLOAT32 || newType == Type.FLOAT64) { - return newSchema; - } - break; + return commonSchemaForInt32(previous, newSchema, newType); case INT64: - if (newType == Type.INT8 || newType == Type.INT16 || newType == Type.INT32) { - return previous; - } - if (newType == Type.FLOAT32 || newType == Type.FLOAT64) { - return newSchema; - } - break; + return commonSchemaForInt64(previous, newSchema, newType); case FLOAT32: - if (newType == Type.INT8 || newType == Type.INT16 || newType == Type.INT32 || newType == Type.INT64) { - return previous; - } - if (newType == Type.FLOAT64) { - return newSchema; - } - break; + return commonSchemaForFloat32(previous, newSchema, newType); case FLOAT64: - if (newType == Type.INT8 || newType == Type.INT16 || newType == Type.INT32 || newType == Type.INT64 || newType == - Type.FLOAT32) { - return previous; - } - break; + return commonSchemaForFloat64(previous, newType); } return null; } @@ -1075,6 +1169,59 @@ protected static Schema commonSchemaFor(Schema previous, SchemaAndValue latest) return previous; } + private static Schema commonSchemaForInt8(Schema newSchema, Type newType) { + if (newType == Type.INT16 || newType == Type.INT32 || newType == Type.INT64 + || newType == Type.FLOAT32 || newType == Type.FLOAT64) { + return newSchema; + } + return null; + } + + private static Schema commonSchemaForInt16(Schema previous, Schema newSchema, Type newType) { + if (newType == Type.INT8) { + return previous; + } else if (newType == Type.INT32 || newType == Type.INT64 + || newType == Type.FLOAT32 || newType == Type.FLOAT64) { + return newSchema; + } + return null; + } + + private static Schema commonSchemaForInt32(Schema previous, Schema newSchema, Type newType) { + if (newType == Type.INT8 || newType == Type.INT16) { + return previous; + } else if (newType == Type.INT64 || newType == Type.FLOAT32 || newType == Type.FLOAT64) { + return newSchema; + } + return null; + } + + private static Schema commonSchemaForInt64(Schema previous, Schema newSchema, Type newType) { + if (newType == Type.INT8 || newType == Type.INT16 || newType == Type.INT32) { + return previous; + } else if (newType == Type.FLOAT32 || newType == Type.FLOAT64) { + return newSchema; + } + return null; + } + + private static Schema commonSchemaForFloat32(Schema previous, Schema newSchema, Type newType) { + if (newType == Type.INT8 || newType == Type.INT16 || newType == Type.INT32 || newType == Type.INT64) { + return previous; + } else if (newType == Type.FLOAT64) { + return newSchema; + } + return null; + } + + private static Schema commonSchemaForFloat64(Schema previous, Type newType) { + if (newType == Type.INT8 || newType == Type.INT16 || newType == Type.INT32 || newType == Type.INT64 + || newType == Type.FLOAT32) { + return previous; + } + return null; + } + protected static List alignListEntriesWithSchema(Schema schema, List input) { Schema valueSchema = schema.valueSchema(); List result = new ArrayList<>(); diff --git a/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java b/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java index b895ed398586d..efa56aca4692d 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java @@ -46,7 +46,7 @@ public void testReconfigureStopException() { private static class TestConnector extends Connector { - private boolean stopException; + private final boolean stopException; private int order = 0; public int stopOrder = -1; public int configureOrder = -1; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java index 3e0c9de8d4c7b..32e304c218ac9 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaProjectorTest.java @@ -82,7 +82,7 @@ public void testNumericTypeProjection() { expectedProjected.put(values[2], Arrays.asList(32767, 32767L, 32767.F, 32767.)); expectedProjected.put(values[3], Arrays.asList(327890L, 327890.F, 327890.)); expectedProjected.put(values[4], Arrays.asList(1.2F, 1.2)); - expectedProjected.put(values[5], Arrays.asList(1.2345)); + expectedProjected.put(values[5], Collections.singletonList(1.2345)); Object promoted; for (int i = 0; i < promotableSchemas.length; ++i) { diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java index 3aad588fc220b..df3c2ade5dd83 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java @@ -81,6 +81,20 @@ public class ValuesTest { INT_LIST.add(-987654321); } + @Test + public void shouldParseNullString() { + SchemaAndValue schemaAndValue = Values.parseString(null); + assertNull(schemaAndValue.schema()); + assertNull(schemaAndValue.value()); + } + + @Test + public void shouldParseEmptyString() { + SchemaAndValue schemaAndValue = Values.parseString(""); + assertEquals(Schema.STRING_SCHEMA, schemaAndValue.schema()); + assertEquals("", schemaAndValue.value()); + } + @Test @Timeout(5) public void shouldNotEncounterInfiniteLoop() { @@ -246,6 +260,20 @@ public void shouldEscapeStringsWithEmbeddedQuotesAndBackslashes() { @Test public void shouldConvertNullValue() { + assertRoundTrip(Schema.INT8_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_INT8_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.INT16_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_INT16_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.INT32_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_INT32_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.INT64_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_INT64_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.FLOAT32_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_FLOAT32_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.FLOAT64_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_FLOAT64_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.BOOLEAN_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_BOOLEAN_SCHEMA, Schema.STRING_SCHEMA, null); assertRoundTrip(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA, null); assertRoundTrip(Schema.OPTIONAL_STRING_SCHEMA, Schema.STRING_SCHEMA, null); } @@ -253,14 +281,22 @@ public void shouldConvertNullValue() { @Test public void shouldConvertBooleanValues() { assertRoundTrip(Schema.BOOLEAN_SCHEMA, Schema.BOOLEAN_SCHEMA, Boolean.FALSE); + assertShortCircuit(Schema.BOOLEAN_SCHEMA, Boolean.FALSE); SchemaAndValue resultFalse = roundTrip(Schema.BOOLEAN_SCHEMA, "false"); assertEquals(Schema.BOOLEAN_SCHEMA, resultFalse.schema()); assertEquals(Boolean.FALSE, resultFalse.value()); + resultFalse = roundTrip(Schema.BOOLEAN_SCHEMA, "0"); + assertEquals(Schema.BOOLEAN_SCHEMA, resultFalse.schema()); + assertEquals(Boolean.FALSE, resultFalse.value()); assertRoundTrip(Schema.BOOLEAN_SCHEMA, Schema.BOOLEAN_SCHEMA, Boolean.TRUE); + assertShortCircuit(Schema.BOOLEAN_SCHEMA, Boolean.TRUE); SchemaAndValue resultTrue = roundTrip(Schema.BOOLEAN_SCHEMA, "true"); assertEquals(Schema.BOOLEAN_SCHEMA, resultTrue.schema()); assertEquals(Boolean.TRUE, resultTrue.value()); + resultTrue = roundTrip(Schema.BOOLEAN_SCHEMA, "1"); + assertEquals(Schema.BOOLEAN_SCHEMA, resultTrue.schema()); + assertEquals(Boolean.TRUE, resultTrue.value()); } @Test @@ -268,6 +304,38 @@ public void shouldFailToParseInvalidBooleanValueString() { assertThrows(DataException.class, () -> Values.convertToBoolean(Schema.STRING_SCHEMA, "\"green\"")); } + @Test + public void shouldConvertInt8() { + assertRoundTrip(Schema.INT8_SCHEMA, Schema.INT8_SCHEMA, (byte) 0); + assertRoundTrip(Schema.INT8_SCHEMA, Schema.INT8_SCHEMA, (byte) 1); + } + + @Test + public void shouldConvertInt64() { + assertRoundTrip(Schema.INT64_SCHEMA, Schema.INT64_SCHEMA, (long) 1); + assertShortCircuit(Schema.INT64_SCHEMA, (long) 1); + } + + @Test + public void shouldConvertFloat32() { + assertRoundTrip(Schema.FLOAT32_SCHEMA, Schema.FLOAT32_SCHEMA, (float) 1); + assertShortCircuit(Schema.FLOAT32_SCHEMA, (float) 1); + } + + @Test + public void shouldConvertFloat64() { + assertRoundTrip(Schema.FLOAT64_SCHEMA, Schema.FLOAT64_SCHEMA, (double) 1); + assertShortCircuit(Schema.FLOAT64_SCHEMA, (double) 1); + } + + @Test + public void shouldConvertEmptyStruct() { + Struct struct = new Struct(SchemaBuilder.struct().build()); + assertThrows(DataException.class, () -> Values.convertToStruct(struct.schema(), null)); + assertThrows(DataException.class, () -> Values.convertToStruct(struct.schema(), "")); + Values.convertToStruct(struct.schema(), struct); + } + @Test public void shouldConvertSimpleString() { assertRoundTrip(Schema.STRING_SCHEMA, "simple"); @@ -361,7 +429,27 @@ public void shouldConvertStringOfListWithOnlyNumericElementTypesIntoListOfLarges assertEquals(3, list.size()); assertEquals(1, ((Number) list.get(0)).intValue()); assertEquals(2, ((Number) list.get(1)).intValue()); - assertEquals(thirdValue, ((Number) list.get(2)).intValue()); + assertEquals(thirdValue, list.get(2)); + } + + @Test + public void shouldConvertIntegralTypesToFloat() { + float thirdValue = Float.MAX_VALUE; + List list = Values.convertToList(Schema.STRING_SCHEMA, "[1, 2, " + thirdValue + "]"); + assertEquals(3, list.size()); + assertEquals(1, ((Number) list.get(0)).intValue()); + assertEquals(2, ((Number) list.get(1)).intValue()); + assertEquals(thirdValue, list.get(2)); + } + + @Test + public void shouldConvertIntegralTypesToDouble() { + double thirdValue = Double.MAX_VALUE; + List list = Values.convertToList(Schema.STRING_SCHEMA, "[1, 2, " + thirdValue + "]"); + assertEquals(3, list.size()); + assertEquals(1, ((Number) list.get(0)).intValue()); + assertEquals(2, ((Number) list.get(1)).intValue()); + assertEquals(thirdValue, list.get(2)); } /** @@ -434,6 +522,34 @@ private void assertParseStringMapWithNoSchema(Map expected, Stri assertEquals(expected, list); } + @Test + public void shouldParseNestedArray() { + SchemaAndValue schemaAndValue = Values.parseString("[[]]"); + assertEquals(Type.ARRAY, schemaAndValue.schema().type()); + assertEquals(Type.ARRAY, schemaAndValue.schema().valueSchema().type()); + } + + @Test + public void shouldParseArrayContainingMap() { + SchemaAndValue schemaAndValue = Values.parseString("[{}]"); + assertEquals(Type.ARRAY, schemaAndValue.schema().type()); + assertEquals(Type.MAP, schemaAndValue.schema().valueSchema().type()); + } + + @Test + public void shouldParseNestedMap() { + SchemaAndValue schemaAndValue = Values.parseString("{\"a\":{}}"); + assertEquals(Type.MAP, schemaAndValue.schema().type()); + assertEquals(Type.MAP, schemaAndValue.schema().valueSchema().type()); + } + + @Test + public void shouldParseMapContainingArray() { + SchemaAndValue schemaAndValue = Values.parseString("{\"a\":[]}"); + assertEquals(Type.MAP, schemaAndValue.schema().type()); + assertEquals(Type.ARRAY, schemaAndValue.schema().valueSchema().type()); + } + /** * We can't infer or successfully parse into a different type, so this returns the same string. */ @@ -445,6 +561,22 @@ public void shouldParseStringListWithExtraDelimitersAndReturnString() { assertEquals(str, result.value()); } + @Test + public void shouldParseStringListWithNullLastAsString() { + String str = "[1, null]"; + SchemaAndValue result = Values.parseString(str); + assertEquals(Type.STRING, result.schema().type()); + assertEquals(str, result.value()); + } + + @Test + public void shouldParseStringListWithNullFirstAsString() { + String str = "[null, 1]"; + SchemaAndValue result = Values.parseString(str); + assertEquals(Type.STRING, result.schema().type()); + assertEquals(str, result.value()); + } + @Test public void shouldParseTimestampStringAsTimestamp() throws Exception { String str = "2019-08-23T14:34:54.346Z"; @@ -585,6 +717,13 @@ public void shouldParseTimeStringAsTimeInMap() throws Exception { assertEquals(Collections.singletonMap(keyStr, expected), result.value()); } + @Test + public void shouldFailToConvertNullTime() { + assertThrows(DataException.class, () -> Values.convertToTime(null, null)); + assertThrows(DataException.class, () -> Values.convertToDate(null, null)); + assertThrows(DataException.class, () -> Values.convertToTimestamp(null, null)); + } + /** * This is technically invalid JSON, and we don't want to simply ignore the blank elements. */ @@ -802,6 +941,51 @@ public void shouldConvertDecimalValues() { assertEquals(value, Values.convertToDecimal(null, buffer, 1)); } + @Test + public void shouldFailToConvertNullToDecimal() { + assertThrows(DataException.class, () -> Values.convertToDecimal(null, null, 1)); + } + + @Test + public void shouldInferByteSchema() { + byte[] bytes = new byte[1]; + Schema byteSchema = Values.inferSchema(bytes); + assertEquals(Schema.BYTES_SCHEMA, byteSchema); + Schema byteBufferSchema = Values.inferSchema(ByteBuffer.wrap(bytes)); + assertEquals(Schema.BYTES_SCHEMA, byteBufferSchema); + } + + @Test + public void shouldInferStructSchema() { + Struct struct = new Struct(SchemaBuilder.struct().build()); + Schema structSchema = Values.inferSchema(struct); + assertEquals(struct.schema(), structSchema); + } + + @Test + public void shouldInferNoSchemaForEmptyList() { + Schema listSchema = Values.inferSchema(Collections.emptyList()); + assertNull(listSchema); + } + + @Test + public void shouldInferNoSchemaForListContainingObject() { + Schema listSchema = Values.inferSchema(Collections.singletonList(new Object())); + assertNull(listSchema); + } + + @Test + public void shouldInferNoSchemaForEmptyMap() { + Schema listSchema = Values.inferSchema(Collections.emptyMap()); + assertNull(listSchema); + } + + @Test + public void shouldInferNoSchemaForMapContainingObject() { + Schema listSchema = Values.inferSchema(Collections.singletonMap(new Object(), new Object())); + assertNull(listSchema); + } + /** * Test parsing distinct number-like types (strings containing numbers, and logical Decimals) in the same list * The parser does not convert Numbers to Decimals, or Strings containing numbers to Numbers automatically. @@ -819,6 +1003,17 @@ public void shouldNotConvertArrayValuesToDecimal() { assertEquals(expected, schemaAndValue.value()); } + @Test + public void shouldParseArrayOfOnlyDecimals() { + List decimals = Arrays.asList(BigDecimal.valueOf(Long.MAX_VALUE).add(BigDecimal.ONE), + BigDecimal.valueOf(Long.MIN_VALUE).subtract(BigDecimal.ONE)); + SchemaAndValue schemaAndValue = Values.parseString(decimals.toString()); + Schema schema = schemaAndValue.schema(); + assertEquals(Type.ARRAY, schema.type()); + assertEquals(Decimal.schema(0), schema.valueSchema()); + assertEquals(decimals, schemaAndValue.value()); + } + @Test public void canConsume() { } @@ -949,6 +1144,16 @@ public void shouldParseDoubleAsFloat64() { assertEquals(value, (Double) schemaAndValue.value(), 0); } + @Test + public void shouldParseFractionalPartsAsIntegerWhenNoFractionalPart() { + assertEquals(new SchemaAndValue(Schema.INT8_SCHEMA, (byte) 1), Values.parseString("1.0")); + assertEquals(new SchemaAndValue(Schema.FLOAT32_SCHEMA, 1.1f), Values.parseString("1.1")); + assertEquals(new SchemaAndValue(Schema.INT16_SCHEMA, (short) 300), Values.parseString("300.0")); + assertEquals(new SchemaAndValue(Schema.FLOAT32_SCHEMA, 300.01f), Values.parseString("300.01")); + assertEquals(new SchemaAndValue(Schema.INT32_SCHEMA, 66000), Values.parseString("66000.0")); + assertEquals(new SchemaAndValue(Schema.FLOAT32_SCHEMA, 66000.0008f), Values.parseString("66000.0008")); + } + protected void assertParsed(String input) { assertParsed(input, input); } @@ -1011,47 +1216,48 @@ protected SchemaAndValue roundTrip(Schema desiredSchema, SchemaAndValue input) { desiredSchema = Values.inferSchema(input); assertNotNull(desiredSchema); } + return convertTo(desiredSchema, serialized); + } + + protected SchemaAndValue convertTo(Schema desiredSchema, Object value) { Object newValue = null; - Schema newSchema = null; switch (desiredSchema.type()) { case STRING: - newValue = Values.convertToString(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToString(Schema.STRING_SCHEMA, value); break; case INT8: - newValue = Values.convertToByte(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToByte(Schema.STRING_SCHEMA, value); break; case INT16: - newValue = Values.convertToShort(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToShort(Schema.STRING_SCHEMA, value); break; case INT32: - newValue = Values.convertToInteger(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToInteger(Schema.STRING_SCHEMA, value); break; case INT64: - newValue = Values.convertToLong(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToLong(Schema.STRING_SCHEMA, value); break; case FLOAT32: - newValue = Values.convertToFloat(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToFloat(Schema.STRING_SCHEMA, value); break; case FLOAT64: - newValue = Values.convertToDouble(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToDouble(Schema.STRING_SCHEMA, value); break; case BOOLEAN: - newValue = Values.convertToBoolean(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToBoolean(Schema.STRING_SCHEMA, value); break; case ARRAY: - newValue = Values.convertToList(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToList(Schema.STRING_SCHEMA, value); break; case MAP: - newValue = Values.convertToMap(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToMap(Schema.STRING_SCHEMA, value); break; case STRUCT: - newValue = Values.convertToStruct(Schema.STRING_SCHEMA, serialized); - break; case BYTES: fail("unexpected schema type"); break; } - newSchema = Values.inferSchema(newValue); + Schema newSchema = Values.inferSchema(newValue); return new SchemaAndValue(newSchema, newValue); } @@ -1075,4 +1281,16 @@ protected void assertRoundTrip(Schema schema, Schema currentSchema, Object value assertEquals(result, result2); } } + + protected void assertShortCircuit(Schema schema, Object value) { + SchemaAndValue result = convertTo(schema, value); + + if (value == null) { + assertNull(result.schema()); + assertNull(result.value()); + } else { + assertEquals(value, result.value()); + assertEquals(schema, result.schema()); + } + } } diff --git a/connect/api/src/test/java/org/apache/kafka/connect/util/ConnectorUtilsTest.java b/connect/api/src/test/java/org/apache/kafka/connect/util/ConnectorUtilsTest.java index b6f96bd4bc15c..1972ff7a89d58 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/util/ConnectorUtilsTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/util/ConnectorUtilsTest.java @@ -33,7 +33,7 @@ public class ConnectorUtilsTest { public void testGroupPartitions() { List> grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 1); - assertEquals(Arrays.asList(FIVE_ELEMENTS), grouped); + assertEquals(Collections.singletonList(FIVE_ELEMENTS), grouped); grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 2); assertEquals(Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5)), grouped); @@ -41,21 +41,21 @@ public void testGroupPartitions() { grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 3); assertEquals(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4), - Arrays.asList(5)), grouped); + Collections.singletonList(5)), grouped); grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 5); - assertEquals(Arrays.asList(Arrays.asList(1), - Arrays.asList(2), - Arrays.asList(3), - Arrays.asList(4), - Arrays.asList(5)), grouped); + assertEquals(Arrays.asList(Collections.singletonList(1), + Collections.singletonList(2), + Collections.singletonList(3), + Collections.singletonList(4), + Collections.singletonList(5)), grouped); grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 7); - assertEquals(Arrays.asList(Arrays.asList(1), - Arrays.asList(2), - Arrays.asList(3), - Arrays.asList(4), - Arrays.asList(5), + assertEquals(Arrays.asList(Collections.singletonList(1), + Collections.singletonList(2), + Collections.singletonList(3), + Collections.singletonList(4), + Collections.singletonList(5), Collections.emptyList(), Collections.emptyList()), grouped); } diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkTaskTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkTaskTest.java index 11106e5a179f9..23a28d8527c18 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkTaskTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSinkTaskTest.java @@ -32,6 +32,7 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -60,7 +61,7 @@ public void testPutFlush() { // We do not call task.start() since it would override the output stream - task.put(Arrays.asList( + task.put(Collections.singletonList( new SinkRecord("topic1", 0, null, null, Schema.STRING_SCHEMA, "line1", 1) )); offsets.put(new TopicPartition("topic1", 0), new OffsetAndMetadata(1L)); @@ -85,7 +86,7 @@ public void testStart() throws IOException { task.start(props); HashMap offsets = new HashMap<>(); - task.put(Arrays.asList( + task.put(Collections.singletonList( new SinkRecord("topic1", 0, null, null, Schema.STRING_SCHEMA, "line0", 1) )); offsets.put(new TopicPartition("topic1", 0), new OffsetAndMetadata(1L)); diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/integration/FileStreamSourceConnectorIntegrationTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/integration/FileStreamSourceConnectorIntegrationTest.java index 95dabf703c585..698f4fcf8d366 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/integration/FileStreamSourceConnectorIntegrationTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/integration/FileStreamSourceConnectorIntegrationTest.java @@ -94,7 +94,7 @@ public void testStopResumeSavedOffset() throws Exception { // Append NUM_LINES more lines to the file try (PrintStream printStream = new PrintStream(Files.newOutputStream(sourceFile.toPath(), StandardOpenOption.APPEND))) { for (int i = NUM_LINES; i < 2 * NUM_LINES; i++) { - printStream.println(String.format(LINE_FORMAT, i)); + printStream.printf(LINE_FORMAT + "%n", i); } } @@ -197,7 +197,7 @@ private File createTempFile(int numLines) throws Exception { try (PrintStream printStream = new PrintStream(Files.newOutputStream(sourceFile.toPath()))) { for (int i = 0; i < numLines; i++) { - printStream.println(String.format(LINE_FORMAT, i)); + printStream.printf(LINE_FORMAT + "%n", i); } } diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java index 8f186400dd29d..353d2eedb9592 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.HashMap; import java.nio.ByteBuffer; +import java.util.Objects; /** Checkpoint records emitted from MirrorCheckpointConnector. Encodes remote consumer group state. */ public class Checkpoint { @@ -180,5 +181,18 @@ byte[] recordKey() { byte[] recordValue() { return serializeValue(VERSION).array(); } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Checkpoint that = (Checkpoint) o; + return upstreamOffset == that.upstreamOffset && downstreamOffset == that.downstreamOffset && Objects.equals(consumerGroupId, that.consumerGroupId) && Objects.equals(topicPartition, that.topicPartition) && Objects.equals(metadata, that.metadata); + } + + @Override + public int hashCode() { + return Objects.hash(consumerGroupId, topicPartition, upstreamOffset, downstreamOffset, metadata); + } } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java new file mode 100644 index 0000000000000..cbe76efecb38c --- /dev/null +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java @@ -0,0 +1,203 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.mirror; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.AuthorizationException; +import org.apache.kafka.common.protocol.types.SchemaException; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.util.Callback; +import org.apache.kafka.connect.util.KafkaBasedLog; +import org.apache.kafka.connect.util.TopicAdmin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static org.apache.kafka.connect.mirror.MirrorCheckpointConfig.CHECKPOINTS_TARGET_CONSUMER_ROLE; + +/** + * Reads once the Kafka log for checkpoints and populates a map of + * checkpoints per consumer group. + * + * The Kafka log is closed after the initial load and only the in memory map is + * used after start. + */ +public class CheckpointStore implements AutoCloseable { + + private static final Logger log = LoggerFactory.getLogger(CheckpointStore.class); + + private final MirrorCheckpointTaskConfig config; + private final Set consumerGroups; + + private TopicAdmin cpAdmin = null; + private KafkaBasedLog backingStore = null; + // accessible for testing + Map> checkpointsPerConsumerGroup; + + private volatile boolean loadSuccess = false; + private volatile boolean isInitialized = false; + + public CheckpointStore(MirrorCheckpointTaskConfig config, Set consumerGroups) { + this.config = config; + this.consumerGroups = new HashSet<>(consumerGroups); + } + + // constructor for testing only + CheckpointStore(Map> checkpointsPerConsumerGroup) { + this.config = null; //ignored by tests + this.consumerGroups = null; //ignored by tests + this.checkpointsPerConsumerGroup = checkpointsPerConsumerGroup; + isInitialized = true; + loadSuccess = true; + } + + // potentially long running + public boolean start() { + checkpointsPerConsumerGroup = readCheckpoints(); + isInitialized = true; + if (log.isTraceEnabled()) { + log.trace("CheckpointStore started, load success={}, map={}", loadSuccess, checkpointsPerConsumerGroup); + } else { + log.debug("CheckpointStore started, load success={}, map.size={}", loadSuccess, checkpointsPerConsumerGroup.size()); + } + return loadSuccess; + } + + public boolean isInitialized() { + return isInitialized; + } + + public void update(String group, Map newCheckpoints) { + Map oldCheckpoints = checkpointsPerConsumerGroup.computeIfAbsent(group, ignored -> new HashMap<>()); + oldCheckpoints.putAll(newCheckpoints); + } + + public Map get(String group) { + Map result = checkpointsPerConsumerGroup.get(group); + return result == null ? null : Collections.unmodifiableMap(result); + } + + public Map> computeConvertedUpstreamOffset() { + Map> result = new HashMap<>(); + + for (Map.Entry> entry : checkpointsPerConsumerGroup.entrySet()) { + String consumerId = entry.getKey(); + Map convertedUpstreamOffset = new HashMap<>(); + for (Checkpoint checkpoint : entry.getValue().values()) { + convertedUpstreamOffset.put(checkpoint.topicPartition(), checkpoint.offsetAndMetadata()); + } + result.put(consumerId, convertedUpstreamOffset); + } + return result; + } + + @Override + public void close() { + releaseResources(); + } + + private void releaseResources() { + Utils.closeQuietly(backingStore != null ? backingStore::stop : null, "backing store for previous Checkpoints"); + Utils.closeQuietly(cpAdmin, "admin client for previous Checkpoints"); + cpAdmin = null; + backingStore = null; + } + + // read the checkpoints topic to initialize the checkpointsPerConsumerGroup state + // the callback may only handle errors thrown by consumer.poll in KafkaBasedLog + // e.g. unauthorized to read from topic (non-retriable) + // if any are encountered, treat the loading of Checkpoints as failed. + private Map> readCheckpoints() { + Map> checkpoints = new HashMap<>(); + Callback> consumedCallback = (error, cpRecord) -> { + if (error != null) { + // if there is no authorization to READ from the topic, we must throw an error + // to stop the KafkaBasedLog forever looping attempting to read to end + checkpoints.clear(); + if (error instanceof RuntimeException) { + throw (RuntimeException) error; + } else { + throw new RuntimeException(error); + } + } else { + try { + Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); + if (consumerGroups.contains(cp.consumerGroupId())) { + Map cps = checkpoints.computeIfAbsent(cp.consumerGroupId(), ignored1 -> new HashMap<>()); + cps.put(cp.topicPartition(), cp); + } + } catch (SchemaException ex) { + log.warn("Ignored invalid checkpoint record at offset {}", cpRecord.offset(), ex); + } + } + }; + + try { + long startTime = System.currentTimeMillis(); + readCheckpointsImpl(config, consumedCallback); + log.debug("starting+stopping KafkaBasedLog took {}ms", System.currentTimeMillis() - startTime); + loadSuccess = true; + } catch (Exception error) { + loadSuccess = false; + if (error instanceof AuthorizationException) { + log.warn("Not authorized to access checkpoints topic {} - " + + "this may degrade offset translation as only checkpoints " + + "for offsets which were mirrored after the task started will be emitted", + config.checkpointsTopic(), error); + } else { + log.info("Exception encountered loading checkpoints topic {} - " + + "this may degrade offset translation as only checkpoints " + + "for offsets which were mirrored after the task started will be emitted", + config.checkpointsTopic(), error); + } + } + return checkpoints; + } + + // accessible for testing + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + try { + cpAdmin = new TopicAdmin( + config.targetAdminConfig("checkpoint-target-admin"), + config.forwardingAdmin(config.targetAdminConfig("checkpoint-target-admin"))); + + backingStore = KafkaBasedLog.withExistingClients( + config.checkpointsTopic(), + MirrorUtils.newConsumer(config.targetConsumerConfig(CHECKPOINTS_TARGET_CONSUMER_ROLE)), + null, + cpAdmin, + consumedCallback, + Time.SYSTEM, + ignored -> { + }, + topicPartition -> topicPartition.partition() == 0); + + backingStore.start(true); + backingStore.stop(); + } finally { + releaseResources(); + } + } +} diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java index e37cee4a79b8b..8be52a9c9be98 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java @@ -75,6 +75,7 @@ public class MirrorCheckpointConfig extends MirrorConnectorConfig { public static final Class GROUP_FILTER_CLASS_DEFAULT = DefaultGroupFilter.class; public static final String OFFSET_SYNCS_SOURCE_CONSUMER_ROLE = "offset-syncs-source-consumer"; public static final String OFFSET_SYNCS_TARGET_CONSUMER_ROLE = "offset-syncs-target-consumer"; + public static final String CHECKPOINTS_TARGET_CONSUMER_ROLE = "checkpoints-target-consumer"; public static final String OFFSET_SYNCS_SOURCE_ADMIN_ROLE = "offset-syncs-source-admin"; public static final String OFFSET_SYNCS_TARGET_ADMIN_ROLE = "offset-syncs-target-admin"; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index 96c287add5f14..7f446efea5df3 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -69,21 +69,25 @@ public class MirrorCheckpointTask extends SourceTask { private MirrorCheckpointMetrics metrics; private Scheduler scheduler; private Map> idleConsumerGroupsOffset; - private Map> checkpointsPerConsumerGroup; + private CheckpointStore checkpointStore; + public MirrorCheckpointTask() {} // for testing MirrorCheckpointTask(String sourceClusterAlias, String targetClusterAlias, - ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, + ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, Set consumerGroups, Map> idleConsumerGroupsOffset, - Map> checkpointsPerConsumerGroup) { + CheckpointStore checkpointStore) { this.sourceClusterAlias = sourceClusterAlias; this.targetClusterAlias = targetClusterAlias; this.replicationPolicy = replicationPolicy; this.offsetSyncStore = offsetSyncStore; + this.consumerGroups = consumerGroups; this.idleConsumerGroupsOffset = idleConsumerGroupsOffset; - this.checkpointsPerConsumerGroup = checkpointsPerConsumerGroup; + this.checkpointStore = checkpointStore; this.topicFilter = topic -> true; + this.interval = Duration.ofNanos(1); + this.pollTimeout = Duration.ofNanos(1); } @Override @@ -103,15 +107,18 @@ public void start(Map props) { targetAdminClient = config.forwardingAdmin(config.targetAdminConfig("checkpoint-target-admin")); metrics = config.metrics(); idleConsumerGroupsOffset = new HashMap<>(); - checkpointsPerConsumerGroup = new HashMap<>(); + checkpointStore = new CheckpointStore(config, consumerGroups); scheduler = new Scheduler(getClass(), config.entityLabel(), config.adminTimeout()); scheduler.execute(() -> { - offsetSyncStore.start(); + // loading the stores are potentially long running operations, so they run asynchronously + // to avoid blocking task::start (until a task has completed starting it cannot be stopped) + boolean checkpointsReadOk = checkpointStore.start(); + offsetSyncStore.start(!checkpointsReadOk); scheduler.scheduleRepeating(this::refreshIdleConsumerGroupOffset, config.syncGroupOffsetsInterval(), "refreshing idle consumers group offsets at target cluster"); scheduler.scheduleRepeatingDelayed(this::syncGroupOffset, config.syncGroupOffsetsInterval(), "sync idle consumer group offset from source to target"); - }, "starting offset sync store"); + }, "starting checkpoint and offset sync stores"); log.info("{} checkpointing {} consumer groups {}->{}: {}.", Thread.currentThread().getName(), consumerGroups.size(), sourceClusterAlias, config.targetClusterAlias(), consumerGroups); } @@ -126,6 +133,7 @@ public void stop() { long start = System.currentTimeMillis(); stopping = true; Utils.closeQuietly(topicFilter, "topic filter"); + Utils.closeQuietly(checkpointStore, "checkpoints store"); Utils.closeQuietly(offsetSyncStore, "offset sync store"); Utils.closeQuietly(sourceAdminClient, "source admin client"); Utils.closeQuietly(targetAdminClient, "target admin client"); @@ -146,8 +154,8 @@ public List poll() throws InterruptedException { while (!stopping && System.currentTimeMillis() < deadline) { Thread.sleep(pollTimeout.toMillis()); } - if (stopping) { - // we are stopping, return early. + if (stopping || !checkpointStore.isInitialized()) { + // we are stopping, or not fully initialized, return early. return null; } List records = new ArrayList<>(); @@ -166,14 +174,13 @@ public List poll() throws InterruptedException { } } - - private List sourceRecordsForGroup(String group) throws InterruptedException { + // visible for testing + List sourceRecordsForGroup(String group) throws InterruptedException { try { long timestamp = System.currentTimeMillis(); Map upstreamGroupOffsets = listConsumerGroupOffsets(group); Map newCheckpoints = checkpointsForGroup(upstreamGroupOffsets, group); - Map oldCheckpoints = checkpointsPerConsumerGroup.computeIfAbsent(group, ignored -> new HashMap<>()); - oldCheckpoints.putAll(newCheckpoints); + checkpointStore.update(group, newCheckpoints); return newCheckpoints.values().stream() .map(x -> checkpointRecord(x, timestamp)) .collect(Collectors.toList()); @@ -195,7 +202,7 @@ Map checkpointsForGroup(Map checkpoints = checkpointsPerConsumerGroup.get(checkpoint.consumerGroupId()); + Map checkpoints = checkpointStore.get(checkpoint.consumerGroupId()); if (checkpoints == null) { log.trace("Emitting {} (first for this group)", checkpoint); return true; @@ -314,7 +321,7 @@ Map> syncGroupOffset() throws Exe Map> offsetToSyncAll = new HashMap<>(); // first, sync offsets for the idle consumers at target - for (Entry> group : getConvertedUpstreamOffset().entrySet()) { + for (Entry> group : checkpointStore.computeConvertedUpstreamOffset().entrySet()) { String consumerGroupId = group.getKey(); // for each idle consumer at target, read the checkpoints (converted upstream offset) // from the pre-populated map @@ -391,18 +398,4 @@ void syncGroupOffset(String consumerGroupId, Map> getConvertedUpstreamOffset() { - Map> result = new HashMap<>(); - - for (Entry> entry : checkpointsPerConsumerGroup.entrySet()) { - String consumerId = entry.getKey(); - Map convertedUpstreamOffset = new HashMap<>(); - for (Checkpoint checkpoint : entry.getValue().values()) { - convertedUpstreamOffset.put(checkpoint.topicPartition(), checkpoint.offsetAndMetadata()); - } - result.put(consumerId, convertedUpstreamOffset); - } - return result; - } } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java index eca5cc68f997f..16038044ddd2b 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java @@ -54,7 +54,7 @@ * started after the position of the consumer group, or if relevant offset syncs for the topic were potentially used as * for translation in an earlier generation of the sync store. */ -class OffsetSyncStore implements AutoCloseable { +public class OffsetSyncStore implements AutoCloseable { private static final Logger log = LoggerFactory.getLogger(OffsetSyncStore.class); // Store one offset sync for each bit of the topic offset. @@ -63,8 +63,10 @@ class OffsetSyncStore implements AutoCloseable { private final KafkaBasedLog backingStore; private final Map offsetSyncs = new ConcurrentHashMap<>(); private final TopicAdmin admin; + protected volatile boolean initializationMustReadToEnd = true; protected volatile boolean readToEnd = false; + // package access to avoid Java 21 "this-escape" warning OffsetSyncStore(MirrorCheckpointConfig config) { Consumer consumer = null; TopicAdmin admin = null; @@ -97,6 +99,7 @@ private KafkaBasedLog createBackingStore(MirrorCheckpointConfig ); } + // for testing OffsetSyncStore() { this.admin = null; this.backingStore = null; @@ -105,12 +108,19 @@ private KafkaBasedLog createBackingStore(MirrorCheckpointConfig /** * Start the OffsetSyncStore, blocking until all previous Offset Syncs have been read from backing storage. */ - public void start() { - backingStore.start(); + public void start(boolean initializationMustReadToEnd) { + this.initializationMustReadToEnd = initializationMustReadToEnd; + log.debug("OffsetSyncStore starting - must read to OffsetSync end = {}", initializationMustReadToEnd); + backingStoreStart(); readToEnd = true; } - OptionalLong translateDownstream(String group, TopicPartition sourceTopicPartition, long upstreamOffset) { + // overridable for testing + void backingStoreStart() { + backingStore.start(false); + } + + public OptionalLong translateDownstream(String group, TopicPartition sourceTopicPartition, long upstreamOffset) { if (!readToEnd) { // If we have not read to the end of the syncs topic at least once, decline to translate any offsets. // This prevents emitting stale offsets while initially reading the offset syncs topic. @@ -214,7 +224,9 @@ private void updateSyncArray(OffsetSync[] syncs, OffsetSync[] original, OffsetSy // While reading to the end of the topic, ensure that our earliest sync is later than // any earlier sync that could have been used for translation, to preserve monotonicity // If the upstream offset rewinds, all previous offsets are invalid, so overwrite them all. - if (!readToEnd || syncs[0].upstreamOffset() > upstreamOffset) { + boolean onlyLoadLastOffset = !readToEnd && initializationMustReadToEnd; + boolean upstreamRewind = upstreamOffset < syncs[0].upstreamOffset(); + if (onlyLoadLastOffset || upstreamRewind) { clearSyncArray(syncs, offsetSync); return; } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/MirrorRestServer.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/MirrorRestServer.java index a5abeff40ce17..7d24a5f14db9c 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/MirrorRestServer.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/MirrorRestServer.java @@ -26,7 +26,6 @@ import org.glassfish.hk2.utilities.binding.AbstractBinder; import org.glassfish.jersey.server.ResourceConfig; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Map; @@ -48,7 +47,7 @@ public void initializeInternalResources(Map herders) { @Override protected Collection> regularResources() { - return Arrays.asList( + return Collections.singletonList( InternalMirrorResource.class ); } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java new file mode 100644 index 0000000000000..b7b3904899f1a --- /dev/null +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.mirror; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.connect.util.Callback; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class CheckpointStoreTest { + + @Test + public void testReadCheckpointsTopic() { + Set consumerGroups = new HashSet<>(); + consumerGroups.add("group1"); + + MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); + when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); + + try (CheckpointStore store = new CheckpointStore(config, consumerGroups) { + @Override + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 0, 0)); + // this record must be ignored as not part of consumerGroups for task + consumedCallback.onCompletion(null, newCheckpointRecord("group2", "t1", 0, 0, 0)); + // this record must be ignored as malformed + consumedCallback.onCompletion(null, + new ConsumerRecord<>("checkpoint.topic", 0, 0L, new byte[0], new byte[0])); + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 1, 1)); + } + }) { + assertFalse(store.isInitialized()); + + assertTrue(store.start(), "expected start to return success"); + assertTrue(store.isInitialized()); + + Map> expected = new HashMap<>(); + expected.put("group1", Collections.singletonMap(new TopicPartition("t1", 0), + new Checkpoint("group1", new TopicPartition("t1", 0), 1, 1, ""))); + assertEquals(expected, store.checkpointsPerConsumerGroup); + } + } + + @Test + public void testReadCheckpointsTopicError() { + Set consumerGroups = new HashSet<>(); + consumerGroups.add("group1"); + + MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); + when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); + + try (CheckpointStore store = new CheckpointStore(config, consumerGroups) { + @Override + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "topic", 1, 0, 0)); + consumedCallback.onCompletion(new TopicAuthorizationException("test"), null); + } + }) { + + assertFalse(store.start(), "expected start to return failure"); + assertTrue(store.isInitialized()); + assertTrue(store.checkpointsPerConsumerGroup.isEmpty()); + } + } + + ConsumerRecord newCheckpointRecord(String gid, String topic, int partition, long upo, long dwo) { + Checkpoint cp = new Checkpoint(gid, new TopicPartition(topic, partition), upo, dwo, ""); + return new ConsumerRecord<>("checkpoint.topic", 0, 0L, cp.recordKey(), cp.recordValue()); + } +} diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index c7aec3e547881..0afc4f74f2f97 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.mirror; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Collections; import java.util.Optional; @@ -32,14 +33,19 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + public class MirrorCheckpointTaskTest { @Test public void testDownstreamTopicRenaming() { MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), null, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), null, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(Collections.emptyMap())); assertEquals(new TopicPartition("source1.topic3", 4), mirrorCheckpointTask.renameTopicPartition(new TopicPartition("topic3", 4)), "Renaming source1.topic3 failed"); @@ -58,9 +64,10 @@ public void testCheckpoint() { long t2UpstreamOffset = 7L; long t2DownstreamOffset = 8L; OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), + Collections.emptyMap(), new CheckpointStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 2), t1UpstreamOffset, t1DownstreamOffset); offsetSyncStore.sync(new TopicPartition("target2.topic5", 6), t2UpstreamOffset, t2DownstreamOffset); Optional optionalCheckpoint1 = mirrorCheckpointTask.checkpoint("group9", new TopicPartition("topic1", 2), @@ -160,7 +167,8 @@ public void testSyncOffset() throws ExecutionException, InterruptedException { checkpointsPerConsumerGroup.put(consumer2, checkpointMapC2); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), null, idleConsumerGroupsOffset, checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, + new CheckpointStore(checkpointsPerConsumerGroup)); Map> output = mirrorCheckpointTask.syncGroupOffset(); @@ -190,7 +198,8 @@ public void testSyncOffsetForTargetGroupWithNullOffsetAndMetadata() throws Execu checkpointsPerConsumerGroup.put(consumer, checkpointMap); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source", "target", - new DefaultReplicationPolicy(), null, idleConsumerGroupsOffset, checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, + new CheckpointStore(checkpointsPerConsumerGroup)); Map> output = mirrorCheckpointTask.syncGroupOffset(); @@ -200,9 +209,10 @@ public void testSyncOffsetForTargetGroupWithNullOffsetAndMetadata() throws Execu @Test public void testNoCheckpointForTopicWithoutOffsetSyncs() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 0), 3L, 4L); Optional checkpoint1 = mirrorCheckpointTask.checkpoint("group9", new TopicPartition("topic1", 1), @@ -216,9 +226,10 @@ public void testNoCheckpointForTopicWithoutOffsetSyncs() { @Test public void testNoCheckpointForTopicWithNullOffsetAndMetadata() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 0), 1L, 3L); Optional checkpoint = mirrorCheckpointTask.checkpoint("g1", new TopicPartition("topic1", 0), null); assertFalse(checkpoint.isPresent()); @@ -227,10 +238,11 @@ public void testNoCheckpointForTopicWithNullOffsetAndMetadata() { @Test public void testCheckpointRecordsMonotonicIfStoreRewinds() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.start(true); Map> checkpointsPerConsumerGroup = new HashMap<>(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(checkpointsPerConsumerGroup)); TopicPartition tp = new TopicPartition("topic1", 0); TopicPartition targetTP = new TopicPartition("source1.topic1", 0); @@ -252,7 +264,7 @@ public void testCheckpointRecordsMonotonicIfStoreRewinds() { offsetSyncStore.sync(tp, upstream++, downstream++); offsetSyncStore.sync(tp, upstream++, downstream++); offsetSyncStore.sync(tp, upstream++, downstream++); - offsetSyncStore.sync(tp, upstream++, downstream++); + offsetSyncStore.sync(tp, upstream, downstream); // The OffsetSyncStore will change its translation of the same offset assertNotEquals(OptionalLong.of(expectedDownstreamOffset), offsetSyncStore.translateDownstream("g1", tp, consumerGroupOffset)); // But the task will filter this out and not emit a checkpoint @@ -271,4 +283,93 @@ private Map assertCheckpointForTopic( assertEquals(truth, checkpoints.containsKey(remoteTp), "should" + (truth ? "" : " not") + " emit offset sync"); return checkpoints; } + + @Test + public void testCheckpointsTaskRestartUsesExistingCheckpoints() { + TopicPartition t1p0 = new TopicPartition("t1", 0); + TopicPartition sourceT1p0 = new TopicPartition("source1.t1", 0); + OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + // OffsetSyncStore contains entries for: 100->100, 200->200, 300->300 + for (int i = 100; i <= 300; i += 100) { + sync(t1p0, i, i); + } + } + }; + offsetSyncStore.start(false); + + MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(Collections.emptyMap())); + + // Generate a checkpoint for upstream offset 250, and assert it maps to downstream 201 + // (as nearest mapping in OffsetSyncStore is 200->200) + Map upstreamGroupOffsets = new HashMap<>(); + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(250)); + Map checkpoints = mirrorCheckpointTask.checkpointsForGroup(upstreamGroupOffsets, "group1"); + assertEquals(1, checkpoints.size()); + assertEquals(new Checkpoint("group1", sourceT1p0, 250, 201, ""), checkpoints.get(sourceT1p0)); + + // Simulate task restart, during which more offsets are added to the sync topic, and thus the + // corresponding OffsetSyncStore no longer has a mapping for 100->100 + // Now OffsetSyncStore contains entries for: 175->175, 375->375, 475->475 + OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore2 = new OffsetSyncStoreTest.FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + for (int i = 175; i <= 475; i += 100) { + sync(t1p0, i, i); + } + } + }; + offsetSyncStore2.start(false); + + // Simulate loading existing checkpoints into checkpointsPerConsumerGroup (250->201) + Map> checkpointsPerConsumerGroup = new HashMap<>(); + checkpointsPerConsumerGroup.put("group1", checkpoints); + MirrorCheckpointTask mirrorCheckpointTask2 = new MirrorCheckpointTask("source1", "target2", + new DefaultReplicationPolicy(), offsetSyncStore2, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(checkpointsPerConsumerGroup)); + + // Upstream offsets 250 and 370 now have the closest downstream value of 176, but this is + // earlier than the downstream value of the last checkpoint (201) - so they are not emitted. + assertEquals(OptionalLong.of(176), offsetSyncStore2.translateDownstream(null, t1p0, 250)); + assertEquals(OptionalLong.of(176), offsetSyncStore2.translateDownstream(null, t1p0, 370)); + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(250)); + assertTrue(mirrorCheckpointTask2.checkpointsForGroup(upstreamGroupOffsets, "group1").isEmpty()); + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(370)); + assertTrue(mirrorCheckpointTask2.checkpointsForGroup(upstreamGroupOffsets, "group1").isEmpty()); + + // Upstream offset 400 has a closes downstream value of 376, and is emitted because it has + // a later downstream offset than the last checkpoint's downstream (201) + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(400)); + Map checkpoints2 = mirrorCheckpointTask2.checkpointsForGroup(upstreamGroupOffsets, "group1"); + assertEquals(1, checkpoints2.size()); + assertEquals(new Checkpoint("group1", sourceT1p0, 400, 376, ""), checkpoints2.get(sourceT1p0)); + } + + @Test + public void testCheckpointStoreInitialized() throws InterruptedException { + CheckpointStore checkpointStore = mock(CheckpointStore.class); + + MirrorCheckpointTask task = new MirrorCheckpointTask("source1", "target2", + new DefaultReplicationPolicy(), + new OffsetSyncStoreTest.FakeOffsetSyncStore(), + Collections.singleton("group"), + Collections.emptyMap(), + checkpointStore) { + + @Override + List sourceRecordsForGroup(String group) { + SourceRecord sr = new SourceRecord(Collections.emptyMap(), Collections.emptyMap(), "", 0, null, null); + return Collections.singletonList(sr); + } + }; + + assertNull(task.poll()); + + when(checkpointStore.isInitialized()).thenReturn(true); + List polled = task.poll(); + assertEquals(1, polled.size()); + } } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java index bc76a1994db92..3f2ddbc62e932 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java @@ -34,18 +34,22 @@ public class OffsetSyncStoreTest { static TopicPartition tp = new TopicPartition("topic1", 2); static class FakeOffsetSyncStore extends OffsetSyncStore { + private boolean startCalled = false; - FakeOffsetSyncStore() { - super(); + @Override + public void start(boolean initializationMustReadToEnd) { + startCalled = true; + super.start(initializationMustReadToEnd); } @Override - public void start() { - // do not call super to avoid NPE without a KafkaBasedLog. - readToEnd = true; + void backingStoreStart() { + // do not start KafkaBasedLog } + // simulate OffsetSync load as from KafkaBasedLog void sync(TopicPartition topicPartition, long upstreamOffset, long downstreamOffset) { + assertTrue(startCalled); // sync in tests should only be called after store.start OffsetSync offsetSync = new OffsetSync(topicPartition, upstreamOffset, downstreamOffset); byte[] key = offsetSync.recordKey(); byte[] value = offsetSync.recordValue(); @@ -57,7 +61,7 @@ void sync(TopicPartition topicPartition, long upstreamOffset, long downstreamOff @Test public void testOffsetTranslation() { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.start(); + store.start(true); // Emit synced downstream offset without dead-reckoning store.sync(tp, 100, 200); @@ -82,20 +86,24 @@ public void testOffsetTranslation() { @Test public void testNoTranslationIfStoreNotStarted() { - try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { + try (FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + // read a sync during startup + sync(tp, 100, 200); + assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 0)); + assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 100)); + assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 200)); + } + }) { // no offsets exist and store is not started assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 100)); assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 200)); - // read a sync during startup - store.sync(tp, 100, 200); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 100)); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 200)); - // After the store is started all offsets are visible - store.start(); + store.start(true); + assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 0)); assertEquals(OptionalLong.of(200), store.translateDownstream(null, tp, 100)); assertEquals(OptionalLong.of(201), store.translateDownstream(null, tp, 200)); @@ -105,26 +113,29 @@ public void testNoTranslationIfStoreNotStarted() { @Test public void testNoTranslationIfNoOffsetSync() { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.start(); + store.start(true); assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); } } @Test public void testPastOffsetTranslation() { - try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - int maxOffsetLag = 10; - int offset = 0; - for (; offset <= 1000; offset += maxOffsetLag) { - store.sync(tp, offset, offset); - assertSparseSyncInvariant(store, tp); + int maxOffsetLag = 10; + try (FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + for (int offset = 0; offset <= 1000; offset += maxOffsetLag) { + sync(tp, offset, offset); + assertSparseSyncInvariant(this, tp); + } } - store.start(); + }) { + store.start(true); // After starting but before seeing new offsets, only the latest startup offset can be translated assertSparseSync(store, 1000, -1); - for (; offset <= 10000; offset += maxOffsetLag) { + for (int offset = 1000 + maxOffsetLag; offset <= 10000; offset += maxOffsetLag) { store.sync(tp, offset, offset); assertSparseSyncInvariant(store, tp); } @@ -155,6 +166,55 @@ public void testPastOffsetTranslation() { } } + // this test has been written knowing the exact offsets syncs stored + @Test + public void testPastOffsetTranslationWithoutInitializationReadToEnd() { + final int maxOffsetLag = 10; + + try (FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + for (int offset = 0; offset <= 1000; offset += maxOffsetLag) { + sync(tp, offset, offset); + assertSparseSyncInvariant(this, tp); + } + } + }) { + + store.start(false); + + // After starting but before seeing new offsets + assertSparseSync(store, 480, 0); + assertSparseSync(store, 720, 480); + assertSparseSync(store, 1000, 990); + + for (int offset = 1000; offset <= 10000; offset += maxOffsetLag) { + store.sync(tp, offset, offset); + assertSparseSyncInvariant(store, tp); + } + + // After seeing new offsets, 1000 was kicked out of the store, so + // offsets before 3840 can only be translated to 1, only previously stored offset is 0 + assertSparseSync(store, 3840, 0); + assertSparseSync(store, 7680, 3840); + assertSparseSync(store, 8640, 7680); + assertSparseSync(store, 9120, 8640); + assertSparseSync(store, 9600, 9120); + assertSparseSync(store, 9840, 9600); + assertSparseSync(store, 9900, 9840); + assertSparseSync(store, 9960, 9900); + assertSparseSync(store, 9990, 9960); + assertSparseSync(store, 10000, 9990); + + // Rewinding upstream offsets should clear all historical syncs + store.sync(tp, 1500, 11000); + assertSparseSyncInvariant(store, tp); + assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); + assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); + assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); + } + } + @Test public void testConsistentlySpacedSyncs() { // Under normal operation, the incoming syncs will be regularly spaced and the store should keep a set of syncs @@ -215,7 +275,7 @@ public void testDroppedSyncsSpacing() { */ private void assertSyncSpacingHasBoundedExpirations(long firstOffset, LongStream steps, int maximumExpirations) { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.start(); + store.start(true); store.sync(tp, firstOffset, firstOffset); PrimitiveIterator.OfLong iterator = steps.iterator(); long offset = firstOffset; diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeLocalMetadataStore.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeLocalMetadataStore.java index 830ddb32eb190..99706d4eaeaac 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeLocalMetadataStore.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeLocalMetadataStore.java @@ -34,8 +34,8 @@ public class FakeLocalMetadataStore { private static final Logger log = LoggerFactory.getLogger(FakeLocalMetadataStore.class); - private static ConcurrentHashMap> allTopics = new ConcurrentHashMap<>(); - private static ConcurrentHashMap> allAcls = new ConcurrentHashMap<>(); + private static final ConcurrentHashMap> ALL_TOPICS = new ConcurrentHashMap<>(); + private static final ConcurrentHashMap> ALL_ACLS = new ConcurrentHashMap<>(); /** * Add topic to allTopics. @@ -44,7 +44,7 @@ public class FakeLocalMetadataStore { public static void addTopicToLocalMetadataStore(NewTopic newTopic) { ConcurrentHashMap configs = new ConcurrentHashMap<>(newTopic.configs()); configs.putIfAbsent("partitions", String.valueOf(newTopic.numPartitions())); - allTopics.putIfAbsent(newTopic.name(), configs); + ALL_TOPICS.putIfAbsent(newTopic.name(), configs); } /** @@ -53,9 +53,9 @@ public static void addTopicToLocalMetadataStore(NewTopic newTopic) { * @param newPartitionCount new partition count. */ public static void updatePartitionCount(String topic, int newPartitionCount) { - ConcurrentHashMap configs = FakeLocalMetadataStore.allTopics.getOrDefault(topic, new ConcurrentHashMap<>()); + ConcurrentHashMap configs = FakeLocalMetadataStore.ALL_TOPICS.getOrDefault(topic, new ConcurrentHashMap<>()); configs.compute("partitions", (key, value) -> String.valueOf(newPartitionCount)); - FakeLocalMetadataStore.allTopics.putIfAbsent(topic, configs); + FakeLocalMetadataStore.ALL_TOPICS.putIfAbsent(topic, configs); } /** @@ -64,7 +64,7 @@ public static void updatePartitionCount(String topic, int newPartitionCount) { * @param newConfig topic config */ public static void updateTopicConfig(String topic, Config newConfig) { - ConcurrentHashMap topicConfigs = FakeLocalMetadataStore.allTopics.getOrDefault(topic, new ConcurrentHashMap<>()); + ConcurrentHashMap topicConfigs = FakeLocalMetadataStore.ALL_TOPICS.getOrDefault(topic, new ConcurrentHashMap<>()); newConfig.entries().stream().forEach(configEntry -> { if (configEntry.name() != null) { if (configEntry.value() != null) { @@ -75,7 +75,7 @@ public static void updateTopicConfig(String topic, Config newConfig) { } } }); - FakeLocalMetadataStore.allTopics.putIfAbsent(topic, topicConfigs); + FakeLocalMetadataStore.ALL_TOPICS.putIfAbsent(topic, topicConfigs); } /** @@ -84,7 +84,7 @@ public static void updateTopicConfig(String topic, Config newConfig) { * @return true if topic name is a key in allTopics */ public static Boolean containsTopic(String topic) { - return allTopics.containsKey(topic); + return ALL_TOPICS.containsKey(topic); } /** @@ -93,7 +93,7 @@ public static Boolean containsTopic(String topic) { * @return topic configurations. */ public static Map topicConfig(String topic) { - return allTopics.getOrDefault(topic, new ConcurrentHashMap<>()); + return ALL_TOPICS.getOrDefault(topic, new ConcurrentHashMap<>()); } /** @@ -102,7 +102,7 @@ public static Map topicConfig(String topic) { * @return {@link List} */ public static List aclBindings(String aclPrinciple) { - return FakeLocalMetadataStore.allAcls.getOrDefault("User:" + aclPrinciple, new Vector<>()); + return FakeLocalMetadataStore.ALL_ACLS.getOrDefault("User:" + aclPrinciple, new Vector<>()); } /** @@ -111,16 +111,16 @@ public static List aclBindings(String aclPrinciple) { * @param aclBinding {@link AclBinding} */ public static void addACLs(String principal, AclBinding aclBinding) { - Vector aclBindings = FakeLocalMetadataStore.allAcls.getOrDefault(principal, new Vector<>()); + Vector aclBindings = FakeLocalMetadataStore.ALL_ACLS.getOrDefault(principal, new Vector<>()); aclBindings.add(aclBinding); - FakeLocalMetadataStore.allAcls.putIfAbsent(principal, aclBindings); + FakeLocalMetadataStore.ALL_ACLS.putIfAbsent(principal, aclBindings); } /** * clear allTopics and allAcls. */ public static void clear() { - allTopics.clear(); - allAcls.clear(); + ALL_TOPICS.clear(); + ALL_ACLS.clear(); } } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsWithCustomForwardingAdminIntegrationTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsWithCustomForwardingAdminIntegrationTest.java index b54aa7073ce88..70f1cd6f6a343 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsWithCustomForwardingAdminIntegrationTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsWithCustomForwardingAdminIntegrationTest.java @@ -164,7 +164,7 @@ public void startClusters() throws Exception { startClusters(additionalConfig); try (Admin adminClient = primary.kafka().createAdminClient()) { - adminClient.createAcls(Arrays.asList( + adminClient.createAcls(Collections.singletonList( new AclBinding( new ResourcePattern(ResourceType.TOPIC, "*", PatternType.LITERAL), new AccessControlEntry("User:connector", "*", AclOperation.ALL, AclPermissionType.ALLOW) @@ -172,7 +172,7 @@ public void startClusters() throws Exception { )).all().get(); } try (Admin adminClient = backup.kafka().createAdminClient()) { - adminClient.createAcls(Arrays.asList( + adminClient.createAcls(Collections.singletonList( new AclBinding( new ResourcePattern(ResourceType.TOPIC, "*", PatternType.LITERAL), new AccessControlEntry("User:connector", "*", AclOperation.ALL, AclPermissionType.ALLOW) @@ -293,7 +293,7 @@ public void testSyncTopicConfigUseProvidedForwardingAdmin() throws Exception { public void testSyncTopicACLsUseProvidedForwardingAdmin() throws Exception { mm2Props.put("sync.topic.acls.enabled", "true"); mm2Config = new MirrorMakerConfig(mm2Props); - List aclBindings = Arrays.asList( + List aclBindings = Collections.singletonList( new AclBinding( new ResourcePattern(ResourceType.TOPIC, "test-topic-1", PatternType.LITERAL), new AccessControlEntry("User:dummy", "*", AclOperation.DESCRIBE, AclPermissionType.ALLOW) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/converters/ByteArrayConverter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/ByteArrayConverter.java index 6d17873d07264..ec934ad56cc42 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/converters/ByteArrayConverter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/ByteArrayConverter.java @@ -27,6 +27,7 @@ import org.apache.kafka.connect.storage.ConverterConfig; import org.apache.kafka.connect.storage.HeaderConverter; +import java.nio.ByteBuffer; import java.util.Map; /** @@ -59,10 +60,10 @@ public byte[] fromConnectData(String topic, Schema schema, Object value) { if (schema != null && schema.type() != Schema.Type.BYTES) throw new DataException("Invalid schema type for ByteArrayConverter: " + schema.type().toString()); - if (value != null && !(value instanceof byte[])) + if (value != null && !(value instanceof byte[]) && !(value instanceof ByteBuffer)) throw new DataException("ByteArrayConverter is not compatible with objects of type " + value.getClass()); - return (byte[]) value; + return value instanceof ByteBuffer ? getBytesFromByteBuffer((ByteBuffer) value) : (byte[]) value; } @Override @@ -84,4 +85,15 @@ public SchemaAndValue toConnectHeader(String topic, String headerKey, byte[] val public void close() { // do nothing } + + private byte[] getBytesFromByteBuffer(ByteBuffer byteBuffer) { + if (byteBuffer == null) { + return null; + } + + byteBuffer.rewind(); + byte[] bytes = new byte[byteBuffer.remaining()]; + byteBuffer.get(bytes); + return bytes; + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java index 93ea9499df252..c262fd076a4c9 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java @@ -324,7 +324,7 @@ public > List> transformationS @SuppressWarnings("unchecked") Predicate predicate = Utils.newInstance(getClass(predicatePrefix + "type"), Predicate.class); predicate.configure(originalsWithPrefix(predicatePrefix)); - transformations.add(new TransformationStage<>(predicate, negate == null ? false : Boolean.parseBoolean(negate.toString()), transformation)); + transformations.add(new TransformationStage<>(predicate, negate != null && Boolean.parseBoolean(negate.toString()), transformation)); } else { transformations.add(new TransformationStage<>(transformation)); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestServer.java index 3adbc0f14ec36..fa25a5ea60d72 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestServer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestServer.java @@ -27,6 +27,7 @@ import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.Map; public class ConnectRestServer extends RestServer { @@ -56,7 +57,7 @@ protected Collection> regularResources() { @Override protected Collection> adminResources() { - return Arrays.asList( + return Collections.singletonList( LoggingResource.class ); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java index 9a47a0e7530bb..37673ee05577d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java @@ -190,15 +190,15 @@ private HttpResponse httpRequest(HttpClient client, String url, String me "Unexpected status code when handling forwarded request: " + responseCode); } } catch (IOException | InterruptedException | TimeoutException | ExecutionException e) { - log.error("IO error forwarding REST request: ", e); + log.error("IO error forwarding REST request to {} :", url, e); throw new ConnectRestException(Response.Status.INTERNAL_SERVER_ERROR, "IO Error trying to forward REST request: " + e.getMessage(), e); } catch (ConnectRestException e) { // catching any explicitly thrown ConnectRestException-s to preserve its status code // and to avoid getting it overridden by the more generic catch (Throwable) clause down below - log.error("Error forwarding REST request", e); + log.error("Error forwarding REST request to {} :", url, e); throw e; } catch (Throwable t) { - log.error("Error forwarding REST request", t); + log.error("Error forwarding REST request to {} :", url, t); throw new ConnectRestException(Response.Status.INTERNAL_SERVER_ERROR, "Error trying to forward REST request: " + t.getMessage(), t); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServerConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServerConfig.java index 4b8b5acf93519..9addf52197fd5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServerConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServerConfig.java @@ -207,7 +207,7 @@ public static void addInternalConfig(ConfigDef configDef) { ).define( BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, ConfigDef.Type.STRING, - SslClientAuth.NONE.toString(), + BrokerSecurityConfigs.SSL_CLIENT_AUTH_DEFAULT, in(Utils.enumOptions(SslClientAuth.class)), ConfigDef.Importance.LOW, BrokerSecurityConfigs.SSL_CLIENT_AUTH_DOC); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfos.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfos.java index d0f67386de35c..dd075b5f90df8 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfos.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfos.java @@ -85,17 +85,15 @@ public int hashCode() { @Override public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("[") - .append(name) - .append(",") - .append(errorCount) - .append(",") - .append(groups) - .append(",") - .append(configs) - .append("]"); - return sb.toString(); + return "[" + + name + + "," + + errorCount + + "," + + groups + + "," + + configs + + "]"; } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java index 5cfdf2dd4f00b..0b1a41c212ed9 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java @@ -141,30 +141,28 @@ public int hashCode() { @Override public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("[") - .append(name) - .append(",") - .append(type) - .append(",") - .append(required) - .append(",") - .append(defaultValue) - .append(",") - .append(importance) - .append(",") - .append(documentation) - .append(",") - .append(group) - .append(",") - .append(orderInGroup) - .append(",") - .append(width) - .append(",") - .append(displayName) - .append(",") - .append(dependents) - .append("]"); - return sb.toString(); + return "[" + + name + + "," + + type + + "," + + required + + "," + + defaultValue + + "," + + importance + + "," + + documentation + + "," + + group + + "," + + orderInGroup + + "," + + width + + "," + + displayName + + "," + + dependents + + "]"; } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java index ff4215d1122b4..a5528730e22fc 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java @@ -87,19 +87,17 @@ public int hashCode() { @Override public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("[") - .append(name) - .append(",") - .append(value) - .append(",") - .append(recommendedValues) - .append(",") - .append(errors) - .append(",") - .append(visible) - .append("]"); - return sb.toString(); + return "[" + + name + + "," + + value + + "," + + recommendedValues + + "," + + errors + + "," + + visible + + "]"; } } 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 36203399766c3..c1f19e3349101 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 @@ -107,6 +107,8 @@ public class KafkaBasedLog { private boolean stopRequested; private final Queue> readLogEndOffsetCallbacks; private final java.util.function.Consumer initializer; + // initialized as false for backward compatibility + private volatile boolean reportErrorsToCallback = false; /** * Create a new KafkaBasedLog object. This does not start reading the log and writing is not permitted until @@ -243,7 +245,12 @@ public void stop() { } public void start() { - log.info("Starting KafkaBasedLog with topic " + topic); + start(false); + } + + public void start(boolean reportErrorsToCallback) { + this.reportErrorsToCallback = reportErrorsToCallback; + log.info("Starting KafkaBasedLog with topic {} reportErrorsToCallback={}", topic, reportErrorsToCallback); // Create the topic admin client and initialize the topic ... admin = topicAdminSupplier.get(); // may be null @@ -468,6 +475,9 @@ private void poll(long timeoutMs) { throw e; } catch (KafkaException e) { log.error("Error polling: " + e); + if (reportErrorsToCallback) { + consumedCallback.onCompletion(e, null); + } } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/converters/ByteArrayConverterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/ByteArrayConverterTest.java index e971a3151c70e..748fd70e63fae 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/converters/ByteArrayConverterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/converters/ByteArrayConverterTest.java @@ -24,6 +24,7 @@ import org.junit.Before; import org.junit.Test; +import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.Collections; @@ -76,6 +77,29 @@ public void testFromConnectNull() { assertNull(converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, null)); } + @Test + public void testFromConnectByteBufferValue() { + ByteBuffer buffer = ByteBuffer.wrap(SAMPLE_BYTES); + assertArrayEquals( + SAMPLE_BYTES, + converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, buffer)); + + buffer.rewind(); + buffer.get(); // Move the position + assertArrayEquals( + SAMPLE_BYTES, + converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, buffer)); + + buffer = null; + assertNull(converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, buffer)); + + byte[] emptyBytes = new byte[0]; + buffer = ByteBuffer.wrap(emptyBytes); + assertArrayEquals( + emptyBytes, + converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, buffer)); + } + @Test public void testToConnect() { SchemaAndValue data = converter.toConnectData(TOPIC, SAMPLE_BYTES); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorTopicsIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorTopicsIntegrationTest.java index 61be2769f3b24..eb055ab13fb11 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorTopicsIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorTopicsIntegrationTest.java @@ -199,8 +199,8 @@ public void testTopicTrackingResetIsDisabled() throws InterruptedException { connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(SINK_CONNECTOR, NUM_TASKS, "Connector tasks did not start in time."); - connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, Arrays.asList(FOO_TOPIC), - "Active topic set is not: " + Arrays.asList(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR); + connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, Collections.singletonList(FOO_TOPIC), + "Active topic set is not: " + Collections.singletonList(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR); // deleting a connector resets its active topics connect.deleteConnector(FOO_CONNECTOR); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java index ab5b711af0f53..fe63658a7578a 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java @@ -48,7 +48,7 @@ public class TaskHandle { private CountDownLatch recordsRemainingLatch; private CountDownLatch recordsToCommitLatch; private int expectedRecords = -1; - private int expectedCommits = -1; + private final int expectedCommits = -1; public TaskHandle(ConnectorHandle connectorHandle, String taskId, Consumer consumer) { this.taskId = taskId; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java index a0b8cecc8f427..3aeecc1d757c3 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java @@ -134,8 +134,8 @@ public class ErrorHandlingTaskTest { private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); - private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); - private TargetState initialState = TargetState.STARTED; + private final ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private final TargetState initialState = TargetState.STARTED; private Time time; private MockConnectMetrics metrics; @SuppressWarnings("unused") @@ -179,7 +179,7 @@ public class ErrorHandlingTaskTest { private ErrorHandlingMetrics errorHandlingMetrics; - private boolean enableTopicCreation; + private final boolean enableTopicCreation; @Parameterized.Parameters public static Collection parameters() { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java index 597041715b538..c5f9f8314d9ef 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java @@ -150,7 +150,7 @@ public static String currentMetricValueAsString(ConnectMetrics metrics, MetricGr } public static class MockMetricsReporter implements MetricsReporter { - private Map metricsByName = new HashMap<>(); + private final Map metricsByName = new HashMap<>(); private MetricsContext metricsContext; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java index 21b2b10c16e75..75f942a6871d8 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java @@ -141,14 +141,14 @@ public class WorkerSinkTaskTest { private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); - private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); - private ConnectorTaskId taskId1 = new ConnectorTaskId("job", 1); - private TargetState initialState = TargetState.STARTED; + private final ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private final ConnectorTaskId taskId1 = new ConnectorTaskId("job", 1); + private final TargetState initialState = TargetState.STARTED; private MockTime time; private WorkerSinkTask workerTask; @Mock private SinkTask sinkTask; - private ArgumentCaptor sinkTaskContext = ArgumentCaptor.forClass(WorkerSinkTaskContext.class); + private final ArgumentCaptor sinkTaskContext = ArgumentCaptor.forClass(WorkerSinkTaskContext.class); private WorkerConfig workerConfig; private MockConnectMetrics metrics; @Mock @@ -169,7 +169,7 @@ public class WorkerSinkTaskTest { private KafkaConsumer consumer; @Mock private ErrorHandlingMetrics errorHandlingMetrics; - private ArgumentCaptor rebalanceListener = ArgumentCaptor.forClass(ConsumerRebalanceListener.class); + private final ArgumentCaptor rebalanceListener = ArgumentCaptor.forClass(ConsumerRebalanceListener.class); @Rule public final MockitoRule rule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); @@ -684,9 +684,9 @@ public void testPreCommitFailureAfterPartialRevocationAndAssignment() { when(consumer.assignment()) .thenReturn(INITIAL_ASSIGNMENT, INITIAL_ASSIGNMENT) - .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2))) - .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2))) - .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2))) + .thenReturn(new HashSet<>(Collections.singletonList(TOPIC_PARTITION2))) + .thenReturn(new HashSet<>(Collections.singletonList(TOPIC_PARTITION2))) + .thenReturn(new HashSet<>(Collections.singletonList(TOPIC_PARTITION2))) .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))) .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))) .thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3))); @@ -1788,7 +1788,7 @@ private void expectRebalanceAssignmentError(RuntimeException e) { } private void verifyInitializeTask() { - verify(consumer).subscribe(eq(asList(TOPIC)), rebalanceListener.capture()); + verify(consumer).subscribe(eq(Collections.singletonList(TOPIC)), rebalanceListener.capture()); verify(sinkTask).initialize(sinkTaskContext.capture()); verify(sinkTask).start(TASK_PROPS); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index 4579794a2c4ff..b51b84d1ac623 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -2602,7 +2602,7 @@ private void testConnectorGeneratesTooManyTasks(boolean enforced) throws Excepti Map taskConfig = new HashMap<>(); // No warnings or exceptions when a connector generates an empty list of task configs - when(sourceConnector.taskConfigs(1)).thenReturn(Arrays.asList()); + when(sourceConnector.taskConfigs(1)).thenReturn(Collections.emptyList()); try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(Worker.class)) { connectorProps.put(TASKS_MAX_CONFIG, "1"); List> taskConfigs = worker.connectorTaskConfigs(CONNECTOR_ID, new ConnectorConfig(plugins, connectorProps)); @@ -2611,7 +2611,7 @@ private void testConnectorGeneratesTooManyTasks(boolean enforced) throws Excepti } // No warnings or exceptions when a connector generates the maximum permitted number of task configs - when(sourceConnector.taskConfigs(1)).thenReturn(Arrays.asList(taskConfig)); + when(sourceConnector.taskConfigs(1)).thenReturn(Collections.singletonList(taskConfig)); when(sourceConnector.taskConfigs(2)).thenReturn(Arrays.asList(taskConfig, taskConfig)); when(sourceConnector.taskConfigs(3)).thenReturn(Arrays.asList(taskConfig, taskConfig, taskConfig)); try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(Worker.class)) { @@ -2672,7 +2672,7 @@ private void testConnectorGeneratesTooManyTasks(boolean enforced) throws Excepti } // One last sanity check in case the connector is reconfigured and respects tasks.max - when(sourceConnector.taskConfigs(1)).thenReturn(Arrays.asList(taskConfig)); + when(sourceConnector.taskConfigs(1)).thenReturn(Collections.singletonList(taskConfig)); try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(Worker.class)) { connectorProps.put(TASKS_MAX_CONFIG, "1"); List> taskConfigs = worker.connectorTaskConfigs(CONNECTOR_ID, new ConnectorConfig(plugins, connectorProps)); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/ConnectProtocolCompatibilityTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/ConnectProtocolCompatibilityTest.java index 6dcbe6c38e617..38084383e2951 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/ConnectProtocolCompatibilityTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/ConnectProtocolCompatibilityTest.java @@ -98,7 +98,7 @@ public void testEagerToCoopMetadata() { public void testEagerToEagerAssignment() { ConnectProtocol.Assignment assignment = new ConnectProtocol.Assignment( ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L, - Arrays.asList(connectorId1, connectorId3), Arrays.asList(taskId2x0)); + Arrays.asList(connectorId1, connectorId3), Collections.singletonList(taskId2x0)); ByteBuffer leaderBuf = ConnectProtocol.serializeAssignment(assignment); ConnectProtocol.Assignment leaderAssignment = ConnectProtocol.deserializeAssignment(leaderBuf); @@ -110,7 +110,7 @@ public void testEagerToEagerAssignment() { ConnectProtocol.Assignment assignment2 = new ConnectProtocol.Assignment( ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L, - Arrays.asList(connectorId2), Arrays.asList(taskId1x0, taskId3x0)); + Collections.singletonList(connectorId2), Arrays.asList(taskId1x0, taskId3x0)); ByteBuffer memberBuf = ConnectProtocol.serializeAssignment(assignment2); ConnectProtocol.Assignment memberAssignment = ConnectProtocol.deserializeAssignment(memberBuf); @@ -125,7 +125,7 @@ public void testEagerToEagerAssignment() { public void testCoopToCoopAssignment() { ExtendedAssignment assignment = new ExtendedAssignment( CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L, - Arrays.asList(connectorId1, connectorId3), Arrays.asList(taskId2x0), + Arrays.asList(connectorId1, connectorId3), Collections.singletonList(taskId2x0), Collections.emptyList(), Collections.emptyList(), 0); ByteBuffer leaderBuf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false); @@ -138,7 +138,7 @@ public void testCoopToCoopAssignment() { ExtendedAssignment assignment2 = new ExtendedAssignment( CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L, - Arrays.asList(connectorId2), Arrays.asList(taskId1x0, taskId3x0), + Collections.singletonList(connectorId2), Arrays.asList(taskId1x0, taskId3x0), Collections.emptyList(), Collections.emptyList(), 0); ByteBuffer memberBuf = ConnectProtocol.serializeAssignment(assignment2); @@ -155,7 +155,7 @@ public void testCoopToCoopAssignment() { public void testEagerToCoopAssignment() { ConnectProtocol.Assignment assignment = new ConnectProtocol.Assignment( ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L, - Arrays.asList(connectorId1, connectorId3), Arrays.asList(taskId2x0)); + Arrays.asList(connectorId1, connectorId3), Collections.singletonList(taskId2x0)); ByteBuffer leaderBuf = ConnectProtocol.serializeAssignment(assignment); ConnectProtocol.Assignment leaderAssignment = @@ -168,7 +168,7 @@ public void testEagerToCoopAssignment() { ConnectProtocol.Assignment assignment2 = new ConnectProtocol.Assignment( ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L, - Arrays.asList(connectorId2), Arrays.asList(taskId1x0, taskId3x0)); + Collections.singletonList(connectorId2), Arrays.asList(taskId1x0, taskId3x0)); ByteBuffer memberBuf = ConnectProtocol.serializeAssignment(assignment2); ConnectProtocol.Assignment memberAssignment = @@ -184,7 +184,7 @@ public void testEagerToCoopAssignment() { public void testCoopToEagerAssignment() { ExtendedAssignment assignment = new ExtendedAssignment( CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L, - Arrays.asList(connectorId1, connectorId3), Arrays.asList(taskId2x0), + Arrays.asList(connectorId1, connectorId3), Collections.singletonList(taskId2x0), Collections.emptyList(), Collections.emptyList(), 0); ByteBuffer leaderBuf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false); @@ -197,7 +197,7 @@ public void testCoopToEagerAssignment() { ExtendedAssignment assignment2 = new ExtendedAssignment( CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L, - Arrays.asList(connectorId2), Arrays.asList(taskId1x0, taskId3x0), + Collections.singletonList(connectorId2), Arrays.asList(taskId1x0, taskId3x0), Collections.emptyList(), Collections.emptyList(), 0); ByteBuffer memberBuf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment2, false); 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 f4a7cd247bd2b..f69f586bc90cf 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 @@ -331,7 +331,7 @@ public void testJoinAssignment() throws Exception { // Join group and get assignment when(member.memberId()).thenReturn("member"); when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1)); + expectRebalance(1, singletonList(CONN1), singletonList(TASK1)); expectConfigRefreshAndSnapshot(SNAPSHOT); ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); @@ -355,7 +355,7 @@ public void testRebalance() throws Exception { // Join group and get assignment when(member.memberId()).thenReturn("member"); when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1)); + expectRebalance(1, singletonList(CONN1), singletonList(TASK1)); expectConfigRefreshAndSnapshot(SNAPSHOT); ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); @@ -380,8 +380,8 @@ public void testRebalance() throws Exception { verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED)); // Rebalance and get a new assignment - expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), ConnectProtocol.Assignment.NO_ERROR, - 1, Arrays.asList(CONN1), Arrays.asList()); + expectRebalance(singletonList(CONN1), singletonList(TASK1), ConnectProtocol.Assignment.NO_ERROR, + 1, singletonList(CONN1), Collections.emptyList()); herder.tick(); time.sleep(3000L); assertStatistics(3, 2, 100, 3000); @@ -414,7 +414,7 @@ public void testIncrementalCooperativeRebalanceForNewMember() throws Exception { // The new member got its assignment expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, - 1, Arrays.asList(CONN1), Arrays.asList(TASK1), 0); + 1, singletonList(CONN1), singletonList(TASK1), 0); // and the new assignment started ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); @@ -445,7 +445,7 @@ public void testIncrementalCooperativeRebalanceForExistingMember() { // Join group. First rebalance contains revocations because a new member joined. when(member.memberId()).thenReturn("member"); when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V1); - expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), + expectRebalance(singletonList(CONN1), singletonList(TASK1), ConnectProtocol.Assignment.NO_ERROR, 1, Collections.emptyList(), Collections.emptyList(), 0); doNothing().when(member).requestRejoin(); @@ -482,7 +482,7 @@ public void testIncrementalCooperativeRebalanceWithDelay() throws Exception { when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V1); expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, - Collections.emptyList(), Arrays.asList(TASK2), + Collections.emptyList(), singletonList(TASK2), rebalanceDelay); expectConfigRefreshAndSnapshot(SNAPSHOT); @@ -503,7 +503,7 @@ public void testIncrementalCooperativeRebalanceWithDelay() throws Exception { // The member got its assignment and revocation expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, - 1, Arrays.asList(CONN1), Arrays.asList(TASK1), 0); + 1, singletonList(CONN1), singletonList(TASK1), 0); // and the new assignment started ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); @@ -528,7 +528,7 @@ public void testRebalanceFailedConnector() throws Exception { // Join group and get assignment when(member.memberId()).thenReturn("member"); when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1)); + expectRebalance(1, singletonList(CONN1), singletonList(TASK1)); expectConfigRefreshAndSnapshot(SNAPSHOT); ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); @@ -550,8 +550,8 @@ public void testRebalanceFailedConnector() throws Exception { verify(worker).startSourceTask(eq(TASK1), any(), any(), any(), eq(herder), eq(TargetState.STARTED)); // Rebalance and get a new assignment - expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), ConnectProtocol.Assignment.NO_ERROR, - 1, Arrays.asList(CONN1), Arrays.asList()); + expectRebalance(singletonList(CONN1), singletonList(TASK1), ConnectProtocol.Assignment.NO_ERROR, + 1, singletonList(CONN1), Collections.emptyList()); // worker is not running, so we should see no call to connectorTaskConfigs() expectExecuteTaskReconfiguration(false, null, null); @@ -606,7 +606,7 @@ public void revokeAndReassign(boolean incompleteRebalance) throws TimeoutExcepti // Perform a partial re-balance just prior to the revocation // bump the configOffset to trigger reading the config topic to the end configOffset++; - expectRebalance(configOffset, Arrays.asList(), Arrays.asList()); + expectRebalance(configOffset, Collections.emptyList(), Collections.emptyList()); // give it the wrong snapshot, as if we're out of sync/can't reach the broker expectConfigRefreshAndSnapshot(SNAPSHOT); doNothing().when(member).requestRejoin(); @@ -616,9 +616,9 @@ public void revokeAndReassign(boolean incompleteRebalance) throws TimeoutExcepti } // Revoke the connector in the next rebalance - expectRebalance(Arrays.asList(CONN1), Arrays.asList(), - ConnectProtocol.Assignment.NO_ERROR, configOffset, Arrays.asList(), - Arrays.asList()); + expectRebalance(singletonList(CONN1), Collections.emptyList(), + ConnectProtocol.Assignment.NO_ERROR, configOffset, Collections.emptyList(), + Collections.emptyList()); if (incompleteRebalance) { // Same as SNAPSHOT, except with an updated offset @@ -643,7 +643,7 @@ public void revokeAndReassign(boolean incompleteRebalance) throws TimeoutExcepti herder.tick(); // re-assign the connector back to the same worker to ensure state was cleaned up - expectRebalance(configOffset, Arrays.asList(CONN1), Arrays.asList()); + expectRebalance(configOffset, singletonList(CONN1), Collections.emptyList()); herder.tick(); @@ -973,7 +973,7 @@ public void testDestroyConnector() throws Exception { when(member.memberId()).thenReturn("leader"); when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // Start with one connector - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList(), true); + expectRebalance(1, singletonList(CONN1), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); @@ -1006,7 +1006,7 @@ public void testDestroyConnector() throws Exception { doNothing().when(statusBackingStore).deleteTopic(eq(CONN1), eq(FOO_TOPIC)); doNothing().when(statusBackingStore).deleteTopic(eq(CONN1), eq(BAR_TOPIC)); - expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), + expectRebalance(singletonList(CONN1), singletonList(TASK1), ConnectProtocol.Assignment.NO_ERROR, 2, "leader", "leaderUrl", Collections.emptyList(), Collections.emptyList(), 0, true); expectConfigRefreshAndSnapshot(ClusterConfigState.EMPTY); @@ -1533,7 +1533,7 @@ public void testConnectorConfigAdded() throws Exception { // Performs rebalance and gets new assignment expectRebalance(Collections.emptyList(), Collections.emptyList(), - ConnectProtocol.Assignment.NO_ERROR, 1, Arrays.asList(CONN1), Collections.emptyList()); + ConnectProtocol.Assignment.NO_ERROR, 1, singletonList(CONN1), Collections.emptyList()); ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); doAnswer(invocation -> { @@ -1556,7 +1556,7 @@ public void testConnectorConfigUpdate() throws Exception { when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // join - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); + expectRebalance(1, singletonList(CONN1), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); expectMemberPoll(); @@ -1591,7 +1591,7 @@ public void testConnectorConfigUpdateFailedTransformation() throws Exception { WorkerConfigTransformer configTransformer = mock(WorkerConfigTransformer.class); // join - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); + expectRebalance(1, singletonList(CONN1), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); expectMemberPoll(); @@ -1646,7 +1646,7 @@ public void testConnectorPaused() throws Exception { when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // join - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); + expectRebalance(1, singletonList(CONN1), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); expectMemberPoll(); @@ -1683,7 +1683,7 @@ public void testConnectorResumed() throws Exception { when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // start with the connector paused - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); + expectRebalance(1, singletonList(CONN1), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT_PAUSED_CONN1); expectMemberPoll(); @@ -1723,7 +1723,7 @@ public void testConnectorStopped() throws Exception { when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); // join - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); + expectRebalance(1, singletonList(CONN1), Collections.emptyList()); expectConfigRefreshAndSnapshot(SNAPSHOT); expectMemberPoll(); @@ -1976,7 +1976,7 @@ public void testTaskConfigAdded() { // Performs rebalance and gets new assignment expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, Collections.emptyList(), - Arrays.asList(TASK0)); + singletonList(TASK0)); expectConfigRefreshAndSnapshot(SNAPSHOT); when(worker.startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), eq(TargetState.STARTED))).thenReturn(true); @@ -2014,7 +2014,7 @@ public void testJoinLeaderCatchUpFails() throws Exception { before = time.milliseconds(); // After backoff, restart the process and this time succeed - expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1), true); + expectRebalance(1, singletonList(CONN1), singletonList(TASK1), true); expectConfigRefreshAndSnapshot(SNAPSHOT); ArgumentCaptor> onStart = ArgumentCaptor.forClass(Callback.class); @@ -2051,7 +2051,7 @@ public void testJoinLeaderCatchUpRetriesForIncrementalCooperative() throws Excep when(member.memberId()).thenReturn("leader"); when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V1); when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); - expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1), true); + expectRebalance(1, singletonList(CONN1), singletonList(TASK1), true); expectConfigRefreshAndSnapshot(SNAPSHOT); expectMemberPoll(); @@ -2072,7 +2072,7 @@ public void testJoinLeaderCatchUpRetriesForIncrementalCooperative() throws Excep // The leader gets the same assignment after a rebalance is triggered expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, - 1, "leader", "leaderUrl", Arrays.asList(CONN1), Arrays.asList(TASK1), 0, true); + 1, "leader", "leaderUrl", singletonList(CONN1), singletonList(TASK1), 0, true); time.sleep(2000L); assertStatistics(3, 1, 100, 2000); @@ -2106,7 +2106,7 @@ public void testJoinLeaderCatchUpRetriesForIncrementalCooperative() throws Excep // After a few retries succeed to read the log to the end expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, - 1, "leader", "leaderUrl", Arrays.asList(CONN1), Arrays.asList(TASK1), 0, true); + 1, "leader", "leaderUrl", singletonList(CONN1), singletonList(TASK1), 0, true); expectConfigRefreshAndSnapshot(SNAPSHOT); before = time.milliseconds(); @@ -2125,7 +2125,7 @@ public void testJoinLeaderCatchUpFailsForIncrementalCooperative() throws Excepti when(member.memberId()).thenReturn("leader"); when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V1); when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); - expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1), true); + expectRebalance(1, singletonList(CONN1), singletonList(TASK1), true); expectConfigRefreshAndSnapshot(SNAPSHOT); expectMemberPoll(); @@ -2146,7 +2146,7 @@ public void testJoinLeaderCatchUpFailsForIncrementalCooperative() throws Excepti // The leader gets the same assignment after a rebalance is triggered expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, - "leader", "leaderUrl", Arrays.asList(CONN1), Arrays.asList(TASK1), 0, true); + "leader", "leaderUrl", singletonList(CONN1), singletonList(TASK1), 0, true); time.sleep(2000L); assertStatistics(3, 1, 100, 2000); @@ -2190,7 +2190,7 @@ public void testJoinLeaderCatchUpFailsForIncrementalCooperative() throws Excepti // The worker gets back the assignment that had given up expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, - 1, "leader", "leaderUrl", Arrays.asList(CONN1), Arrays.asList(TASK1), + 1, "leader", "leaderUrl", singletonList(CONN1), singletonList(TASK1), 0, true); expectConfigRefreshAndSnapshot(SNAPSHOT); @@ -2267,7 +2267,7 @@ public void testAccessors() throws Exception { @Test public void testPutConnectorConfig() throws Exception { when(member.memberId()).thenReturn("leader"); - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList(), true); + expectRebalance(1, singletonList(CONN1), Collections.emptyList(), true); when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); expectConfigRefreshAndSnapshot(SNAPSHOT); when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); @@ -2388,7 +2388,7 @@ public void testPatchConnectorConfigNotALeader() { // Patch the connector config. expectMemberEnsureActive(); - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList(), false); + expectRebalance(1, singletonList(CONN1), Collections.emptyList(), false); FutureCallback> patchCallback = new FutureCallback<>(); herder.patchConnectorConfig(CONN1, new HashMap<>(), patchCallback); @@ -2401,7 +2401,7 @@ public void testPatchConnectorConfigNotALeader() { @Test public void testPatchConnectorConfig() throws Exception { when(member.memberId()).thenReturn("leader"); - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList(), true); + expectRebalance(1, singletonList(CONN1), Collections.emptyList(), true); when(statusBackingStore.connectors()).thenReturn(Collections.emptySet()); Map originalConnConfig = new HashMap<>(CONN1_CONFIG); @@ -2440,7 +2440,7 @@ public void testPatchConnectorConfig() throws Exception { patchedConnConfig.put("foo3", "added"); expectMemberEnsureActive(); - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList(), true); + expectRebalance(1, singletonList(CONN1), Collections.emptyList(), true); ArgumentCaptor> validateCallback = ArgumentCaptor.forClass(Callback.class); doAnswer(invocation -> { @@ -2567,7 +2567,7 @@ public void testPutTaskConfigsSignatureNotRequiredV0() { verify(member).wakeup(); verifyNoMoreInteractions(member, taskConfigCb); assertEquals( - Arrays.asList("awaiting startup"), + singletonList("awaiting startup"), stages ); } @@ -2584,7 +2584,7 @@ public void testPutTaskConfigsSignatureNotRequiredV1() { verify(member).wakeup(); verifyNoMoreInteractions(member, taskConfigCb); assertEquals( - Arrays.asList("awaiting startup"), + singletonList("awaiting startup"), stages ); } @@ -2690,7 +2690,7 @@ public void testPutTaskConfigsValidRequiredSignature() { verifyNoMoreInteractions(member, taskConfigCb); assertEquals( - Arrays.asList("awaiting startup"), + singletonList("awaiting startup"), stages ); } @@ -3369,7 +3369,7 @@ public void testPollDurationOnSlowConnectorOperations() { public void shouldThrowWhenStartAndStopExecutorThrowsRejectedExecutionExceptionAndHerderNotStopping() { when(member.memberId()).thenReturn("leader"); when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList(), true); + expectRebalance(1, singletonList(CONN1), Collections.emptyList(), true); expectConfigRefreshAndSnapshot(SNAPSHOT); herder.startAndStopExecutor.shutdown(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignorTest.java index 3edb4d52dc0ba..319bdc9f9f8ef 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignorTest.java @@ -659,7 +659,7 @@ public void testAssignConnectorsWhenBalanced() { .collect(Collectors.toList()); expectedAssignment.get(0).connectors().addAll(Arrays.asList("connector6", "connector9")); expectedAssignment.get(1).connectors().addAll(Arrays.asList("connector7", "connector10")); - expectedAssignment.get(2).connectors().addAll(Arrays.asList("connector8")); + expectedAssignment.get(2).connectors().add("connector8"); List newConnectors = newConnectors(6, 11); assignor.assignConnectors(existingAssignment, newConnectors); @@ -679,11 +679,11 @@ public void testAssignTasksWhenBalanced() { expectedAssignment.get(0).connectors().addAll(Arrays.asList("connector6", "connector9")); expectedAssignment.get(1).connectors().addAll(Arrays.asList("connector7", "connector10")); - expectedAssignment.get(2).connectors().addAll(Arrays.asList("connector8")); + expectedAssignment.get(2).connectors().add("connector8"); expectedAssignment.get(0).tasks().addAll(Arrays.asList(new ConnectorTaskId("task", 6), new ConnectorTaskId("task", 9))); expectedAssignment.get(1).tasks().addAll(Arrays.asList(new ConnectorTaskId("task", 7), new ConnectorTaskId("task", 10))); - expectedAssignment.get(2).tasks().addAll(Arrays.asList(new ConnectorTaskId("task", 8))); + expectedAssignment.get(2).tasks().add(new ConnectorTaskId("task", 8)); List newConnectors = newConnectors(6, 11); assignor.assignConnectors(existingAssignment, newConnectors); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorIncrementalTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorIncrementalTest.java index 8b28c37aca92a..ca5c3bdc6f833 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorIncrementalTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorIncrementalTest.java @@ -75,18 +75,18 @@ public class WorkerCoordinatorIncrementalTest { @Rule public MockitoRule rule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); - private String connectorId1 = "connector1"; - private String connectorId2 = "connector2"; - private ConnectorTaskId taskId1x0 = new ConnectorTaskId(connectorId1, 0); - private ConnectorTaskId taskId2x0 = new ConnectorTaskId(connectorId2, 0); - - private String groupId = "test-group"; - private int sessionTimeoutMs = 10; - private int rebalanceTimeoutMs = 60; - private int heartbeatIntervalMs = 2; - private long retryBackoffMs = 100; - private long retryBackoffMaxMs = 1000; - private int requestTimeoutMs = 1000; + private final String connectorId1 = "connector1"; + private final String connectorId2 = "connector2"; + private final ConnectorTaskId taskId1x0 = new ConnectorTaskId(connectorId1, 0); + private final ConnectorTaskId taskId2x0 = new ConnectorTaskId(connectorId2, 0); + + private final String groupId = "test-group"; + private final int sessionTimeoutMs = 10; + private final int rebalanceTimeoutMs = 60; + private final int heartbeatIntervalMs = 2; + private final long retryBackoffMs = 100; + private final long retryBackoffMaxMs = 1000; + private final int requestTimeoutMs = 1000; private MockTime time; private MockClient client; private Node node; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/ConnectRestServerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/ConnectRestServerTest.java index 72a51afd500be..0494d00272668 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/ConnectRestServerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/ConnectRestServerTest.java @@ -72,7 +72,7 @@ public class ConnectRestServerTest { @Mock private Plugins plugins; private ConnectRestServer server; private CloseableHttpClient httpClient; - private Collection responses = new ArrayList<>(); + private final Collection responses = new ArrayList<>(); protected static final String KAFKA_CLUSTER_ID = "Xbafgnagvar"; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerConfigTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerConfigTest.java index 28dd725afd455..4930c1a3ef221 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerConfigTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerConfigTest.java @@ -21,6 +21,7 @@ import org.junit.Test; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -68,7 +69,7 @@ public void testListenersConfigAllowedValues() { props.put(RestServerConfig.LISTENERS_CONFIG, "http://a.b:9999"); config = RestServerConfig.forPublic(null, props); - assertEquals(Arrays.asList("http://a.b:9999"), config.listeners()); + assertEquals(Collections.singletonList("http://a.b:9999"), config.listeners()); props.put(RestServerConfig.LISTENERS_CONFIG, "http://a.b:9999, https://a.b:7812"); config = RestServerConfig.forPublic(null, props); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java index 07d48a6e01f8a..4b3f6e673238b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java @@ -60,13 +60,13 @@ public class FileOffsetBackingStoreTest { private Converter converter; - private static Map firstSet = new HashMap<>(); + private static final Map FIRST_SET = new HashMap<>(); private static final Runnable EMPTY_RUNNABLE = () -> { }; static { - firstSet.put(buffer("key"), buffer("value")); - firstSet.put(null, null); + FIRST_SET.put(buffer("key"), buffer("value")); + FIRST_SET.put(null, null); } @Before @@ -96,7 +96,7 @@ public void testGetSet() throws Exception { @SuppressWarnings("unchecked") Callback setCallback = mock(Callback.class); - store.set(firstSet, setCallback).get(); + store.set(FIRST_SET, setCallback).get(); Map values = store.get(Arrays.asList(buffer("key"), buffer("bad"))).get(); assertEquals(buffer("value"), values.get(buffer("key"))); @@ -109,7 +109,7 @@ public void testSaveRestore() throws Exception { @SuppressWarnings("unchecked") Callback setCallback = mock(Callback.class); - store.set(firstSet, setCallback).get(); + store.set(FIRST_SET, setCallback).get(); store.stop(); // Restore into a new store to ensure correct reload from scratch diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreMockitoTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreMockitoTest.java index 6c9057a35177c..3ec037734f116 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreMockitoTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreMockitoTest.java @@ -61,6 +61,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -159,7 +160,7 @@ public class KafkaConfigBackingStoreMockitoTest { new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1)), new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(2)) ); - + private static final Struct TARGET_STATE_STARTED = new Struct(KafkaConfigBackingStore.TARGET_STATE_V0).put("state", "STARTED"); private static final Struct TARGET_STATE_PAUSED = new Struct(KafkaConfigBackingStore.TARGET_STATE_V1) .put("state", "PAUSED") .put("state.v2", "PAUSED"); @@ -800,7 +801,7 @@ public void testRestoreZeroTasks() { // Should see a single connector and its config should be the last one seen anywhere in the log ClusterConfigState configState = configStorage.snapshot(); assertEquals(8, configState.offset()); // Should always be next to be read, even if uncommitted - assertEquals(Arrays.asList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors())); + assertEquals(Collections.singletonList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors())); // CONNECTOR_CONFIG_STRUCTS[2] -> SAMPLE_CONFIGS[2] assertEquals(SAMPLE_CONFIGS.get(2), configState.connectorConfig(CONNECTOR_IDS.get(0))); // Should see 0 tasks for that connector. @@ -1053,7 +1054,7 @@ public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exceptio // After reading the log, it should have been in an inconsistent state ClusterConfigState configState = configStorage.snapshot(); assertEquals(6, configState.offset()); // Should always be next to be read, not last committed - assertEquals(Arrays.asList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors())); + assertEquals(Collections.singletonList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors())); // Inconsistent data should leave us with no tasks listed for the connector and an entry in the inconsistent list assertEquals(Collections.emptyList(), configState.tasks(CONNECTOR_IDS.get(0))); // Both TASK_CONFIG_STRUCTS[0] -> SAMPLE_CONFIGS[0] @@ -1086,8 +1087,8 @@ public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exceptio // This is only two more ahead of the last one because multiple calls fail, and so their configs are not written // to the topic. Only the last call with 1 task config + 1 commit actually gets written. assertEquals(8, configState.offset()); - assertEquals(Arrays.asList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors())); - assertEquals(Arrays.asList(TASK_IDS.get(0)), configState.tasks(CONNECTOR_IDS.get(0))); + assertEquals(Collections.singletonList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors())); + assertEquals(Collections.singletonList(TASK_IDS.get(0)), configState.tasks(CONNECTOR_IDS.get(0))); assertEquals(SAMPLE_CONFIGS.get(0), configState.taskConfig(TASK_IDS.get(0))); assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors()); @@ -1184,6 +1185,147 @@ public void testRestoreRestartRequestInconsistentState() { verify(configLog).stop(); } + @Test + public void testPutTaskConfigsZeroTasks() throws Exception { + configStorage.setupAndCreateKafkaBasedLog(TOPIC, config); + verifyConfigure(); + configStorage.start(); + verify(configLog).start(); + + // Records to be read by consumer as it reads to the end of the log + doAnswer(expectReadToEnd(new LinkedHashMap<>())). + doAnswer(expectReadToEnd(Collections.singletonMap(COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)))) + .when(configLog).readToEnd(); + + expectConvertWriteRead( + COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(0), + "tasks", 0); // We have 0 tasks + + // Bootstrap as if we had already added the connector, but no tasks had been added yet + addConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), Collections.emptyList()); + + + // Null before writing + ClusterConfigState configState = configStorage.snapshot(); + assertEquals(-1, configState.offset()); + + // Writing task configs should block until all the writes have been performed and the root record update + // has completed + List> taskConfigs = Collections.emptyList(); + configStorage.putTaskConfigs("connector1", taskConfigs); + + // Validate root config by listing all connectors and tasks + configState = configStorage.snapshot(); + assertEquals(1, configState.offset()); + String connectorName = CONNECTOR_IDS.get(0); + assertEquals(Collections.singletonList(connectorName), new ArrayList<>(configState.connectors())); + assertEquals(Collections.emptyList(), configState.tasks(connectorName)); + assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors()); + + // As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks + verify(configUpdateListener).onTaskConfigUpdate(Collections.emptyList()); + + configStorage.stop(); + verify(configLog).stop(); + } + + @Test + public void testBackgroundUpdateTargetState() throws Exception { + // verify that we handle target state changes correctly when they come up through the log + List> existingRecords = Arrays.asList( + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(1), + CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(3), new RecordHeaders(), Optional.empty())); + LinkedHashMap deserializedOnStartup = new LinkedHashMap<>(); + deserializedOnStartup.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0)); + deserializedOnStartup.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0)); + deserializedOnStartup.put(CONFIGS_SERIALIZED.get(2), TASK_CONFIG_STRUCTS.get(0)); + deserializedOnStartup.put(CONFIGS_SERIALIZED.get(3), TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR); + logOffset = 5; + + expectStart(existingRecords, deserializedOnStartup); + when(configLog.partitionCount()).thenReturn(1); + + configStorage.setupAndCreateKafkaBasedLog(TOPIC, config); + verifyConfigure(); + configStorage.start(); + verify(configLog).start(); + + // Should see a single connector with initial state started + ClusterConfigState configState = configStorage.snapshot(); + assertEquals(Collections.singleton(CONNECTOR_IDS.get(0)), configStorage.connectorTargetStates.keySet()); + assertEquals(TargetState.STARTED, configState.targetState(CONNECTOR_IDS.get(0))); + + LinkedHashMap serializedAfterStartup = new LinkedHashMap<>(); + serializedAfterStartup.put(TARGET_STATE_KEYS.get(0), CONFIGS_SERIALIZED.get(0)); + serializedAfterStartup.put(TARGET_STATE_KEYS.get(1), CONFIGS_SERIALIZED.get(1)); + doAnswer(expectReadToEnd(serializedAfterStartup)).when(configLog).readToEnd(); + + Map deserializedAfterStartup = new HashMap<>(); + deserializedAfterStartup.put(TARGET_STATE_KEYS.get(0), TARGET_STATE_PAUSED); + deserializedAfterStartup.put(TARGET_STATE_KEYS.get(1), TARGET_STATE_STOPPED); + expectRead(serializedAfterStartup, deserializedAfterStartup); + + // Should see two connectors now, one paused and one stopped + configStorage.refresh(0, TimeUnit.SECONDS); + verify(configUpdateListener).onConnectorTargetStateChange(CONNECTOR_IDS.get(0)); + configState = configStorage.snapshot(); + + assertEquals(new HashSet<>(CONNECTOR_IDS), configStorage.connectorTargetStates.keySet()); + assertEquals(TargetState.PAUSED, configState.targetState(CONNECTOR_IDS.get(0))); + assertEquals(TargetState.STOPPED, configState.targetState(CONNECTOR_IDS.get(1))); + + configStorage.stop(); + verify(configStorage).stop(); + } + + @Test + public void testSameTargetState() throws Exception { + // verify that we handle target state changes correctly when they come up through the log + List> existingRecords = Arrays.asList( + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(1), + CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(3), new RecordHeaders(), Optional.empty())); + LinkedHashMap deserialized = new LinkedHashMap<>(); + deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0)); + deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0)); + deserialized.put(CONFIGS_SERIALIZED.get(2), TASK_CONFIG_STRUCTS.get(0)); + deserialized.put(CONFIGS_SERIALIZED.get(3), TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR); + logOffset = 5; + + expectStart(existingRecords, deserialized); + + when(configLog.partitionCount()).thenReturn(1); + + configStorage.setupAndCreateKafkaBasedLog(TOPIC, config); + verifyConfigure(); + configStorage.start(); + verify(configLog).start(); + + ClusterConfigState configState = configStorage.snapshot(); + expectRead(TARGET_STATE_KEYS.get(0), CONFIGS_SERIALIZED.get(0), TARGET_STATE_STARTED); + // Should see a single connector with initial state paused + assertEquals(TargetState.STARTED, configState.targetState(CONNECTOR_IDS.get(0))); + + expectRead(TARGET_STATE_KEYS.get(0), CONFIGS_SERIALIZED.get(0), TARGET_STATE_STARTED); + // on resume update listener shouldn't be called + verify(configUpdateListener, never()).onConnectorTargetStateChange(anyString()); + + configStorage.stop(); + verify(configStorage).stop(); + } + + @Test public void testPutLogLevel() throws Exception { final String logger1 = "org.apache.zookeeper"; @@ -1293,6 +1435,12 @@ private void expectRead(LinkedHashMap serializedValues, } } + private void expectRead(final String key, final byte[] serializedValue, Struct deserializedValue) { + LinkedHashMap serializedData = new LinkedHashMap<>(); + serializedData.put(key, serializedValue); + expectRead(serializedData, Collections.singletonMap(key, deserializedValue)); + } + // This map needs to maintain ordering private Answer> expectReadToEnd(final Map serializedConfigs) { return invocation -> { @@ -1315,4 +1463,11 @@ private Map structToMap(Struct struct) { for (Field field : struct.schema().fields()) result.put(field.name(), struct.get(field)); return result; } + + private void addConnector(String connectorName, Map connectorConfig, List> taskConfigs) { + for (int i = 0; i < taskConfigs.size(); i++) + configStorage.taskConfigs.put(new ConnectorTaskId(connectorName, i), taskConfigs.get(i)); + configStorage.connectorConfigs.put(connectorName, connectorConfig); + configStorage.connectorTaskCounts.put(connectorName, taskConfigs.size()); + } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java index fc2caf75d9b18..2e7b388413c55 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java @@ -28,7 +28,6 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.runtime.TargetState; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.distributed.DistributedConfig; import org.apache.kafka.connect.util.Callback; @@ -52,13 +51,11 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; import java.util.function.Supplier; import static org.apache.kafka.connect.storage.KafkaConfigBackingStore.INCLUDE_TASKS_FIELD_NAME; @@ -151,7 +148,7 @@ public class KafkaConfigBackingStoreTest { private Converter converter; @Mock private ConfigBackingStore.UpdateListener configUpdateListener; - private Map props = new HashMap<>(DEFAULT_CONFIG_STORAGE_PROPS); + private final Map props = new HashMap<>(DEFAULT_CONFIG_STORAGE_PROPS); private DistributedConfig config; @Mock KafkaBasedLog storeLog; @@ -328,7 +325,7 @@ public void testPutTaskConfigs() throws Exception { configState = configStorage.snapshot(); assertEquals(3, configState.offset()); String connectorName = CONNECTOR_IDS.get(0); - assertEquals(Arrays.asList(connectorName), new ArrayList<>(configState.connectors())); + assertEquals(Collections.singletonList(connectorName), new ArrayList<>(configState.connectors())); assertEquals(Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)), configState.tasks(connectorName)); assertEquals(SAMPLE_CONFIGS.get(0), configState.taskConfig(TASK_IDS.get(0))); assertEquals(SAMPLE_CONFIGS.get(1), configState.taskConfig(TASK_IDS.get(1))); @@ -378,7 +375,7 @@ public void testPutTaskConfigsStartsOnlyReconfiguredTasks() throws Exception { "tasks", 1); // Starts with 2 tasks, after update has 3 // As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks - configUpdateListener.onTaskConfigUpdate(Arrays.asList(TASK_IDS.get(2))); + configUpdateListener.onTaskConfigUpdate(Collections.singletonList(TASK_IDS.get(2))); EasyMock.expectLastCall(); // Records to be read by consumer as it reads to the end of the log @@ -430,167 +427,6 @@ public void testPutTaskConfigsStartsOnlyReconfiguredTasks() throws Exception { PowerMock.verifyAll(); } - @Test - public void testPutTaskConfigsZeroTasks() throws Exception { - expectConfigure(); - expectStart(Collections.emptyList(), Collections.emptyMap()); - - // Task configs should read to end, write to the log, read to end, write root. - expectReadToEnd(new LinkedHashMap<>()); - expectConvertWriteRead( - COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(0), - "tasks", 0); // We have 0 tasks - // As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks - configUpdateListener.onTaskConfigUpdate(Collections.emptyList()); - EasyMock.expectLastCall(); - - // Records to be read by consumer as it reads to the end of the log - LinkedHashMap serializedConfigs = new LinkedHashMap<>(); - serializedConfigs.put(COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)); - expectReadToEnd(serializedConfigs); - - expectPartitionCount(1); - expectStop(); - - PowerMock.replayAll(); - - configStorage.setupAndCreateKafkaBasedLog(TOPIC, config); - configStorage.start(); - - // Bootstrap as if we had already added the connector, but no tasks had been added yet - whiteboxAddConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), Collections.emptyList()); - - // Null before writing - ClusterConfigState configState = configStorage.snapshot(); - assertEquals(-1, configState.offset()); - - // Writing task configs should block until all the writes have been performed and the root record update - // has completed - List> taskConfigs = Collections.emptyList(); - configStorage.putTaskConfigs("connector1", taskConfigs); - - // Validate root config by listing all connectors and tasks - configState = configStorage.snapshot(); - assertEquals(1, configState.offset()); - String connectorName = CONNECTOR_IDS.get(0); - assertEquals(Arrays.asList(connectorName), new ArrayList<>(configState.connectors())); - assertEquals(Collections.emptyList(), configState.tasks(connectorName)); - assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors()); - - configStorage.stop(); - - PowerMock.verifyAll(); - } - - @Test - public void testBackgroundUpdateTargetState() throws Exception { - // verify that we handle target state changes correctly when they come up through the log - - expectConfigure(); - List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), - CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), - CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty()), - new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(1), - CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), - new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), - CONFIGS_SERIALIZED.get(3), new RecordHeaders(), Optional.empty())); - LinkedHashMap deserializedOnStartup = new LinkedHashMap<>(); - deserializedOnStartup.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0)); - deserializedOnStartup.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0)); - deserializedOnStartup.put(CONFIGS_SERIALIZED.get(2), TASK_CONFIG_STRUCTS.get(0)); - deserializedOnStartup.put(CONFIGS_SERIALIZED.get(3), TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR); - logOffset = 5; - - expectStart(existingRecords, deserializedOnStartup); - - LinkedHashMap serializedAfterStartup = new LinkedHashMap<>(); - serializedAfterStartup.put(TARGET_STATE_KEYS.get(0), CONFIGS_SERIALIZED.get(0)); - serializedAfterStartup.put(TARGET_STATE_KEYS.get(1), CONFIGS_SERIALIZED.get(1)); - - Map deserializedAfterStartup = new HashMap<>(); - deserializedAfterStartup.put(TARGET_STATE_KEYS.get(0), TARGET_STATE_PAUSED); - deserializedAfterStartup.put(TARGET_STATE_KEYS.get(1), TARGET_STATE_STOPPED); - - expectRead(serializedAfterStartup, deserializedAfterStartup); - - configUpdateListener.onConnectorTargetStateChange(CONNECTOR_IDS.get(0)); - EasyMock.expectLastCall(); - - expectPartitionCount(1); - expectStop(); - - PowerMock.replayAll(); - - configStorage.setupAndCreateKafkaBasedLog(TOPIC, config); - configStorage.start(); - - // Should see a single connector with initial state started - ClusterConfigState configState = configStorage.snapshot(); - assertEquals(Collections.singleton(CONNECTOR_IDS.get(0)), configStorage.connectorTargetStates.keySet()); - assertEquals(TargetState.STARTED, configState.targetState(CONNECTOR_IDS.get(0))); - - // Should see two connectors now, one paused and one stopped - configStorage.refresh(0, TimeUnit.SECONDS); - configState = configStorage.snapshot(); - assertEquals(new HashSet<>(CONNECTOR_IDS), configStorage.connectorTargetStates.keySet()); - assertEquals(TargetState.PAUSED, configState.targetState(CONNECTOR_IDS.get(0))); - assertEquals(TargetState.STOPPED, configState.targetState(CONNECTOR_IDS.get(1))); - - configStorage.stop(); - - PowerMock.verifyAll(); - } - - @Test - public void testSameTargetState() throws Exception { - // verify that we handle target state changes correctly when they come up through the log - - expectConfigure(); - List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), - CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), - CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty()), - new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(1), - CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), - new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), - CONFIGS_SERIALIZED.get(3), new RecordHeaders(), Optional.empty())); - LinkedHashMap deserialized = new LinkedHashMap<>(); - deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0)); - deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0)); - deserialized.put(CONFIGS_SERIALIZED.get(2), TASK_CONFIG_STRUCTS.get(0)); - deserialized.put(CONFIGS_SERIALIZED.get(3), TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR); - logOffset = 5; - - expectStart(existingRecords, deserialized); - - // on resume update listener shouldn't be called - configUpdateListener.onConnectorTargetStateChange(EasyMock.anyString()); - EasyMock.expectLastCall().andStubThrow(new AssertionError("unexpected call to onConnectorTargetStateChange")); - - expectRead(TARGET_STATE_KEYS.get(0), CONFIGS_SERIALIZED.get(0), TARGET_STATE_STARTED); - - expectPartitionCount(1); - expectStop(); - - PowerMock.replayAll(); - - configStorage.setupAndCreateKafkaBasedLog(TOPIC, config); - configStorage.start(); - - // Should see a single connector with initial state paused - ClusterConfigState configState = configStorage.snapshot(); - assertEquals(TargetState.STARTED, configState.targetState(CONNECTOR_IDS.get(0))); - - configStorage.refresh(0, TimeUnit.SECONDS); - - configStorage.stop(); - - PowerMock.verifyAll(); - } - private void expectConfigure() throws Exception { PowerMock.expectPrivate(configStorage, "createKafkaBasedLog", EasyMock.capture(capturedTopic), EasyMock.capture(capturedProducerProps), @@ -636,12 +472,6 @@ private void expectRead(LinkedHashMap serializedValues, } } - private void expectRead(final String key, final byte[] serializedValue, Struct deserializedValue) { - LinkedHashMap serializedData = new LinkedHashMap<>(); - serializedData.put(key, serializedValue); - expectRead(serializedData, Collections.singletonMap(key, deserializedValue)); - } - // Expect a conversion & write to the underlying log, followed by a subsequent read when the data is consumed back // from the log. Validate the data that is captured when the conversion is performed matches the specified data // (by checking a single field's value) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java index a0e4d569f403e..b8503ceb83945 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java @@ -110,7 +110,7 @@ public class KafkaOffsetBackingStoreTest { private static final ByteBuffer TP0_VALUE_NEW = buffer("VAL0_NEW"); private static final ByteBuffer TP1_VALUE_NEW = buffer("VAL1_NEW"); - private Map props = new HashMap<>(DEFAULT_PROPS); + private final Map props = new HashMap<>(DEFAULT_PROPS); @Mock KafkaBasedLog storeLog; @Mock diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ConvertingFutureCallbackTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ConvertingFutureCallbackTest.java index 7977a291df6de..b930cec34b573 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ConvertingFutureCallbackTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ConvertingFutureCallbackTest.java @@ -192,9 +192,9 @@ public void shouldNotCancelIfMayNotCancelWhileRunning() throws Exception { } protected static class TestConvertingFutureCallback extends ConvertingFutureCallback { - private AtomicInteger numberOfConversions = new AtomicInteger(); - private CountDownLatch getInvoked = new CountDownLatch(1); - private CountDownLatch cancelInvoked = new CountDownLatch(1); + private final AtomicInteger numberOfConversions = new AtomicInteger(); + private final CountDownLatch getInvoked = new CountDownLatch(1); + private final CountDownLatch cancelInvoked = new CountDownLatch(1); public int numberOfConversions() { return numberOfConversions.get(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TestFuture.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TestFuture.java index 0883040a33f21..9130d8badc9ba 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TestFuture.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TestFuture.java @@ -26,7 +26,7 @@ public class TestFuture implements Future { private volatile boolean resolved; private T result; private Throwable exception; - private CountDownLatch getCalledLatch; + private final CountDownLatch getCalledLatch; private volatile boolean resolveOnGet; private T resolveOnGetResult; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index e5e58f78761b9..f959d225377e9 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -57,6 +57,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.apache.kafka.metadata.BrokerState; import org.apache.kafka.network.SocketServerConfigs; +import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.config.ZkConfigs; import org.apache.kafka.storage.internals.log.CleanerConfig; import org.slf4j.Logger; @@ -163,7 +164,7 @@ public void start() { private void doStart() { brokerConfig.put(ZkConfigs.ZK_CONNECT_CONFIG, zKConnectString()); - putIfAbsent(brokerConfig, KafkaConfig.DeleteTopicEnableProp(), true); + putIfAbsent(brokerConfig, ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, true); putIfAbsent(brokerConfig, GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, 0); putIfAbsent(brokerConfig, GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, (short) brokers.length); putIfAbsent(brokerConfig, AUTO_CREATE_TOPICS_ENABLE_CONFIG, false); @@ -178,7 +179,7 @@ private void doStart() { listenerName = new ListenerName(listenerConfig.toString()); for (int i = 0; i < brokers.length; i++) { - brokerConfig.put(KafkaConfig.BrokerIdProp(), i); + brokerConfig.put(ServerConfigs.BROKER_ID_CONFIG, i); currentBrokerLogDirs[i] = currentBrokerLogDirs[i] == null ? createLogDir() : currentBrokerLogDirs[i]; brokerConfig.put(LOG_DIR_CONFIG, currentBrokerLogDirs[i]); if (!hasListenerConfig) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index 7ba602f9c9b64..ef59c23d76ebf 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -20,8 +20,10 @@ import kafka.cluster.EndPoint; import kafka.cluster.Partition; import kafka.log.UnifiedLog; +import kafka.log.remote.quota.RLMQuotaManager; +import kafka.log.remote.quota.RLMQuotaManagerConfig; import kafka.server.BrokerTopicStats; -import kafka.server.KafkaConfig; +import kafka.server.QuotaType; import kafka.server.StopPartition; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicIdPartition; @@ -30,6 +32,7 @@ import org.apache.kafka.common.errors.OffsetOutOfRangeException; import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.message.FetchResponseData; +import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; @@ -45,6 +48,7 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.common.CheckpointFile; import org.apache.kafka.server.common.OffsetAndEpoch; +import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.log.remote.metadata.storage.ClassLoaderAwareRemoteLogMetadataManager; import org.apache.kafka.server.log.remote.storage.ClassLoaderAwareRemoteStorageManager; import org.apache.kafka.server.log.remote.storage.LogSegmentData; @@ -148,11 +152,15 @@ public class RemoteLogManager implements Closeable { private final Function> fetchLog; private final BiConsumer updateRemoteLogStartOffset; private final BrokerTopicStats brokerTopicStats; + private final Metrics metrics; private final RemoteStorageManager remoteLogStorageManager; private final RemoteLogMetadataManager remoteLogMetadataManager; + private final RLMQuotaManager rlmCopyQuotaManager; + private final RLMQuotaManager rlmFetchQuotaManager; + private final RemoteIndexCache indexCache; private final RemoteStorageThreadPool remoteStorageReaderThreadPool; private final RLMScheduledThreadPool rlmScheduledThreadPool; @@ -170,6 +178,8 @@ public class RemoteLogManager implements Closeable { private Optional endpoint = Optional.empty(); private boolean closed = false; + private volatile boolean remoteLogManagerConfigured = false; + /** * Creates RemoteLogManager instance with the given arguments. * @@ -181,6 +191,7 @@ public class RemoteLogManager implements Closeable { * @param fetchLog function to get UnifiedLog instance for a given topic. * @param updateRemoteLogStartOffset function to update the log-start-offset for a given topic partition. * @param brokerTopicStats BrokerTopicStats instance to update the respective metrics. + * @param metrics Metrics instance */ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, int brokerId, @@ -189,7 +200,8 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, Time time, Function> fetchLog, BiConsumer updateRemoteLogStartOffset, - BrokerTopicStats brokerTopicStats) throws IOException { + BrokerTopicStats brokerTopicStats, + Metrics metrics) throws IOException { this.rlmConfig = rlmConfig; this.brokerId = brokerId; this.logDir = logDir; @@ -198,9 +210,13 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, this.fetchLog = fetchLog; this.updateRemoteLogStartOffset = updateRemoteLogStartOffset; this.brokerTopicStats = brokerTopicStats; + this.metrics = metrics; remoteLogStorageManager = createRemoteStorageManager(); remoteLogMetadataManager = createRemoteLogMetadataManager(); + rlmCopyQuotaManager = createRLMCopyQuotaManager(); + rlmFetchQuotaManager = createRLMFetchQuotaManager(); + indexCache = new RemoteIndexCache(rlmConfig.remoteLogIndexFileCacheTotalSizeBytes(), remoteLogStorageManager, logDir); delayInMs = rlmConfig.remoteLogManagerTaskIntervalMs(); rlmScheduledThreadPool = new RLMScheduledThreadPool(rlmConfig.remoteLogManagerThreadPoolSize()); @@ -228,6 +244,28 @@ private void removeMetrics() { remoteStorageReaderThreadPool.removeMetrics(); } + RLMQuotaManager createRLMCopyQuotaManager() { + return new RLMQuotaManager(copyQuotaManagerConfig(rlmConfig), metrics, QuotaType.RLMCopy$.MODULE$, + "Tracking copy byte-rate for Remote Log Manager", time); + } + + RLMQuotaManager createRLMFetchQuotaManager() { + return new RLMQuotaManager(fetchQuotaManagerConfig(rlmConfig), metrics, QuotaType.RLMFetch$.MODULE$, + "Tracking fetch byte-rate for Remote Log Manager", time); + } + + static RLMQuotaManagerConfig copyQuotaManagerConfig(RemoteLogManagerConfig rlmConfig) { + return new RLMQuotaManagerConfig(rlmConfig.remoteLogManagerCopyMaxBytesPerSecond(), + rlmConfig.remoteLogManagerCopyNumQuotaSamples(), + rlmConfig.remoteLogManagerCopyQuotaWindowSizeSeconds()); + } + + static RLMQuotaManagerConfig fetchQuotaManagerConfig(RemoteLogManagerConfig rlmConfig) { + return new RLMQuotaManagerConfig(rlmConfig.remoteLogManagerFetchMaxBytesPerSecond(), + rlmConfig.remoteLogManagerFetchNumQuotaSamples(), + rlmConfig.remoteLogManagerFetchQuotaWindowSizeSeconds()); + } + private T createDelegate(ClassLoader classLoader, String className) { try { return (T) classLoader.loadClass(className) @@ -257,7 +295,7 @@ public RemoteStorageManager run() { private void configureRSM() { final Map rsmProps = new HashMap<>(rlmConfig.remoteStorageManagerProps()); - rsmProps.put(KafkaConfig.BrokerIdProp(), brokerId); + rsmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); remoteLogStorageManager.configure(rsmProps); } @@ -291,7 +329,7 @@ private void configureRLMM() { // update the remoteLogMetadataProps here to override endpoint config if any rlmmProps.putAll(rlmConfig.remoteLogMetadataManagerProps()); - rlmmProps.put(KafkaConfig.BrokerIdProp(), brokerId); + rlmmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); rlmmProps.put(LOG_DIR_CONFIG, logDir); rlmmProps.put("cluster.id", clusterId); @@ -303,6 +341,11 @@ public void startup() { // in connecting to the brokers or remote storages. configureRSM(); configureRLMM(); + remoteLogManagerConfigured = true; + } + + private boolean isRemoteLogManagerConfigured() { + return this.remoteLogManagerConfigured; } public RemoteStorageManager storageManager() { @@ -336,6 +379,10 @@ public void onLeadershipChange(Set partitionsBecomeLeader, Map topicIds) { LOGGER.debug("Received leadership changes for leaders: {} and followers: {}", partitionsBecomeLeader, partitionsBecomeFollower); + if (this.rlmConfig.enableRemoteStorageSystem() && !isRemoteLogManagerConfigured()) { + throw new KafkaException("RemoteLogManager is not configured when remote storage system is enabled"); + } + Map leaderPartitionsWithLeaderEpoch = filterPartitions(partitionsBecomeLeader) .collect(Collectors.toMap( partition -> new TopicIdPartition(topicIds.get(partition.topic()), partition.topicPartition()), @@ -608,6 +655,7 @@ boolean isLeader() { // the task's run() method. private volatile Optional copiedOffsetOption = Optional.empty(); private volatile boolean isLogStartOffsetUpdatedOnBecomingLeader = false; + private volatile Optional logDirectory = Optional.empty(); public void convertToLeader(int leaderEpochVal) { if (leaderEpochVal < 0) { @@ -818,6 +866,13 @@ public void run() { } UnifiedLog log = unifiedLogOptional.get(); + // In the first run after completing altering logDir within broker, we should make sure the state is reset. (KAFKA-16711) + if (!log.parentDir().equals(logDirectory.orElse(null))) { + copiedOffsetOption = Optional.empty(); + isLogStartOffsetUpdatedOnBecomingLeader = false; + logDirectory = Optional.of(log.parentDir()); + } + if (isLeader()) { // Copy log segments to remote storage copyLogSegmentsToRemote(log); diff --git a/core/src/main/java/kafka/log/remote/quota/RLMQuotaManager.java b/core/src/main/java/kafka/log/remote/quota/RLMQuotaManager.java new file mode 100644 index 0000000000000..e21f00f1f5b0e --- /dev/null +++ b/core/src/main/java/kafka/log/remote/quota/RLMQuotaManager.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.log.remote.quota; + +import kafka.server.QuotaType; +import kafka.server.SensorAccess; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Quota; +import org.apache.kafka.common.metrics.QuotaViolationException; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.SimpleRate; +import org.apache.kafka.common.utils.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.runtime.BoxedUnit; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +public class RLMQuotaManager { + private static final Logger LOGGER = LoggerFactory.getLogger(RLMQuotaManager.class); + + private final RLMQuotaManagerConfig config; + private final Metrics metrics; + private final QuotaType quotaType; + private final String description; + private final Time time; + + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + private final SensorAccess sensorAccess; + private Quota quota; + + public RLMQuotaManager(RLMQuotaManagerConfig config, Metrics metrics, QuotaType quotaType, String description, Time time) { + this.config = config; + this.metrics = metrics; + this.quotaType = quotaType; + this.description = description; + this.time = time; + + this.quota = new Quota(config.quotaBytesPerSecond(), true); + this.sensorAccess = new SensorAccess(lock, metrics); + } + + public void updateQuota(Quota newQuota) { + lock.writeLock().lock(); + try { + this.quota = newQuota; + + Map allMetrics = metrics.metrics(); + MetricName quotaMetricName = metricName(); + KafkaMetric metric = allMetrics.get(quotaMetricName); + if (metric != null) { + LOGGER.info("Sensor for quota-id {} already exists. Setting quota to {} in MetricConfig", quotaMetricName, newQuota); + metric.config(getQuotaMetricConfig(newQuota)); + } + } finally { + lock.writeLock().unlock(); + } + } + + public boolean isQuotaExceeded() { + Sensor sensorInstance = sensor(); + try { + sensorInstance.checkQuotas(); + } catch (QuotaViolationException qve) { + LOGGER.debug("Quota violated for sensor ({}), metric: ({}), metric-value: ({}), bound: ({})", + sensorInstance.name(), qve.metric().metricName(), qve.value(), qve.bound()); + return true; + } + return false; + } + + public void record(double value) { + sensor().record(value, time.milliseconds(), false); + } + + private MetricConfig getQuotaMetricConfig(Quota quota) { + return new MetricConfig() + .timeWindow(config.quotaWindowSizeSeconds(), TimeUnit.SECONDS) + .samples(config.numQuotaSamples()) + .quota(quota); + } + + private MetricName metricName() { + return metrics.metricName("byte-rate", quotaType.toString(), description, Collections.emptyMap()); + } + + private Sensor sensor() { + return sensorAccess.getOrCreate( + quotaType.toString(), + RLMQuotaManagerConfig.INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS, + sensor -> { + sensor.add(metricName(), new SimpleRate(), getQuotaMetricConfig(quota)); + return BoxedUnit.UNIT; + } + ); + } +} \ No newline at end of file diff --git a/core/src/main/java/kafka/log/remote/quota/RLMQuotaManagerConfig.java b/core/src/main/java/kafka/log/remote/quota/RLMQuotaManagerConfig.java new file mode 100644 index 0000000000000..7f1ad2015a64f --- /dev/null +++ b/core/src/main/java/kafka/log/remote/quota/RLMQuotaManagerConfig.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.log.remote.quota; + +public class RLMQuotaManagerConfig { + public static final int INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS = 3600; + + private final long quotaBytesPerSecond; + private final int numQuotaSamples; + private final int quotaWindowSizeSeconds; + + /** + * Configuration settings for quota management + * + * @param quotaBytesPerSecond The quota in bytes per second + * @param numQuotaSamples The number of samples to retain in memory + * @param quotaWindowSizeSeconds The time span of each sample + */ + public RLMQuotaManagerConfig(long quotaBytesPerSecond, int numQuotaSamples, int quotaWindowSizeSeconds) { + this.quotaBytesPerSecond = quotaBytesPerSecond; + this.numQuotaSamples = numQuotaSamples; + this.quotaWindowSizeSeconds = quotaWindowSizeSeconds; + } + + public long quotaBytesPerSecond() { + return quotaBytesPerSecond; + } + + public int numQuotaSamples() { + return numQuotaSamples; + } + + public int quotaWindowSizeSeconds() { + return quotaWindowSizeSeconds; + } + + @Override + public String toString() { + return "RLMQuotaManagerConfig{" + + "quotaBytesPerSecond=" + quotaBytesPerSecond + + ", numQuotaSamples=" + numQuotaSamples + + ", quotaWindowSizeSeconds=" + quotaWindowSizeSeconds + + '}'; + } +} diff --git a/core/src/main/java/kafka/server/ReplicaAlterLogDirsTierStateMachine.java b/core/src/main/java/kafka/server/ReplicaAlterLogDirsTierStateMachine.java deleted file mode 100644 index 8561fae01990e..0000000000000 --- a/core/src/main/java/kafka/server/ReplicaAlterLogDirsTierStateMachine.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server; - -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.message.FetchResponseData.PartitionData; - -import java.util.Optional; - -/** - The replica alter log dirs tier state machine is unsupported but is provided to the ReplicaAlterLogDirsThread. - */ -public class ReplicaAlterLogDirsTierStateMachine implements TierStateMachine { - - public PartitionFetchState start(TopicPartition topicPartition, - PartitionFetchState currentFetchState, - PartitionData fetchPartitionData) throws Exception { - // JBOD is not supported with tiered storage. - throw new UnsupportedOperationException("Building remote log aux state is not supported in ReplicaAlterLogDirsThread."); - } - - public Optional maybeAdvanceState(TopicPartition topicPartition, - PartitionFetchState currentFetchState) { - return Optional.empty(); - } -} diff --git a/core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java b/core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java deleted file mode 100644 index 0462e12c05bca..0000000000000 --- a/core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java +++ /dev/null @@ -1,280 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server; - -import java.io.BufferedReader; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.StandardCopyOption; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -import kafka.cluster.Partition; -import kafka.log.UnifiedLog; -import kafka.log.remote.RemoteLogManager; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.message.FetchResponseData.PartitionData; -import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset; -import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition; -import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.server.common.CheckpointFile; -import org.apache.kafka.server.common.OffsetAndEpoch; -import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; -import org.apache.kafka.server.log.remote.storage.RemoteStorageException; -import org.apache.kafka.server.log.remote.storage.RemoteStorageManager; -import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile; -import org.apache.kafka.storage.internals.log.EpochEntry; -import org.apache.kafka.storage.internals.log.LogFileUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import scala.Option; -import scala.collection.JavaConverters; - -import static org.apache.kafka.storage.internals.log.LogStartOffsetIncrementReason.LeaderOffsetIncremented; - -/** - The replica fetcher tier state machine follows a state machine progression. - - Currently, the tier state machine follows a synchronous execution, and we only need to start the machine. - There is no need to advance the state. - - When started, the tier state machine will fetch the local log start offset of the - leader and then build the follower's remote log aux state until the leader's - local log start offset. - */ -public class ReplicaFetcherTierStateMachine implements TierStateMachine { - private static final Logger log = LoggerFactory.getLogger(ReplicaFetcherTierStateMachine.class); - - private final LeaderEndPoint leader; - private final ReplicaManager replicaMgr; - - public ReplicaFetcherTierStateMachine(LeaderEndPoint leader, - ReplicaManager replicaMgr) { - this.leader = leader; - this.replicaMgr = replicaMgr; - } - - - /** - * Start the tier state machine for the provided topic partition. Currently, this start method will build the - * entire remote aux log state synchronously. - * - * @param topicPartition the topic partition - * @param currentFetchState the current PartitionFetchState which will - * be used to derive the return value - * @param fetchPartitionData the data from the fetch response that returned the offset moved to tiered storage error - * - * @return the new PartitionFetchState after the successful start of the - * tier state machine - */ - public PartitionFetchState start(TopicPartition topicPartition, - PartitionFetchState currentFetchState, - PartitionData fetchPartitionData) throws Exception { - - OffsetAndEpoch epochAndLeaderLocalStartOffset = leader.fetchEarliestLocalOffset(topicPartition, currentFetchState.currentLeaderEpoch()); - int epoch = epochAndLeaderLocalStartOffset.leaderEpoch(); - long leaderLocalStartOffset = epochAndLeaderLocalStartOffset.offset(); - - long offsetToFetch = 0; - replicaMgr.brokerTopicStats().topicStats(topicPartition.topic()).buildRemoteLogAuxStateRequestRate().mark(); - replicaMgr.brokerTopicStats().allTopicsStats().buildRemoteLogAuxStateRequestRate().mark(); - - try { - offsetToFetch = buildRemoteLogAuxState(topicPartition, currentFetchState.currentLeaderEpoch(), leaderLocalStartOffset, epoch, fetchPartitionData.logStartOffset()); - } catch (RemoteStorageException e) { - replicaMgr.brokerTopicStats().topicStats(topicPartition.topic()).failedBuildRemoteLogAuxStateRate().mark(); - replicaMgr.brokerTopicStats().allTopicsStats().failedBuildRemoteLogAuxStateRate().mark(); - throw e; - } - - OffsetAndEpoch fetchLatestOffsetResult = leader.fetchLatestOffset(topicPartition, currentFetchState.currentLeaderEpoch()); - long leaderEndOffset = fetchLatestOffsetResult.offset(); - - long initialLag = leaderEndOffset - offsetToFetch; - - return PartitionFetchState.apply(currentFetchState.topicId(), offsetToFetch, Option.apply(initialLag), currentFetchState.currentLeaderEpoch(), - Fetching$.MODULE$, replicaMgr.localLogOrException(topicPartition).latestEpoch()); - } - - /** - * This is currently a no-op but will be used for implementing async tiering logic in KAFKA-13560. - * - * @param topicPartition the topic partition - * @param currentFetchState the current PartitionFetchState which will - * be used to derive the return value - * - * @return the original PartitionFetchState - */ - public Optional maybeAdvanceState(TopicPartition topicPartition, - PartitionFetchState currentFetchState) { - // No-op for now - return Optional.of(currentFetchState); - } - - private EpochEndOffset fetchEarlierEpochEndOffset(Integer epoch, - TopicPartition partition, - Integer currentLeaderEpoch) { - int previousEpoch = epoch - 1; - - // Find the end-offset for the epoch earlier to the given epoch from the leader - Map partitionsWithEpochs = new HashMap<>(); - partitionsWithEpochs.put(partition, new OffsetForLeaderPartition().setPartition(partition.partition()).setCurrentLeaderEpoch(currentLeaderEpoch).setLeaderEpoch(previousEpoch)); - Option maybeEpochEndOffset = leader.fetchEpochEndOffsets(JavaConverters.mapAsScalaMap(partitionsWithEpochs)).get(partition); - if (maybeEpochEndOffset.isEmpty()) { - throw new KafkaException("No response received for partition: " + partition); - } - - EpochEndOffset epochEndOffset = maybeEpochEndOffset.get(); - if (epochEndOffset.errorCode() != Errors.NONE.code()) { - throw Errors.forCode(epochEndOffset.errorCode()).exception(); - } - - return epochEndOffset; - } - - private List readLeaderEpochCheckpoint(RemoteLogManager rlm, - RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws IOException, RemoteStorageException { - InputStream inputStream = rlm.storageManager().fetchIndex(remoteLogSegmentMetadata, RemoteStorageManager.IndexType.LEADER_EPOCH); - try (BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))) { - CheckpointFile.CheckpointReadBuffer readBuffer = new CheckpointFile.CheckpointReadBuffer<>("", bufferedReader, 0, LeaderEpochCheckpointFile.FORMATTER); - return readBuffer.read(); - } - } - - private void buildProducerSnapshotFile(File snapshotFile, - RemoteLogSegmentMetadata remoteLogSegmentMetadata, - RemoteLogManager rlm) throws IOException, RemoteStorageException { - File tmpSnapshotFile = new File(snapshotFile.getAbsolutePath() + ".tmp"); - // Copy it to snapshot file in atomic manner. - Files.copy(rlm.storageManager().fetchIndex(remoteLogSegmentMetadata, RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT), - tmpSnapshotFile.toPath(), StandardCopyOption.REPLACE_EXISTING); - Utils.atomicMoveWithFallback(tmpSnapshotFile.toPath(), snapshotFile.toPath(), false); - } - - /** - * It tries to build the required state for this partition from leader and remote storage so that it can start - * fetching records from the leader. The return value is the next offset to fetch from the leader, which is the - * next offset following the end offset of the remote log portion. - */ - private Long buildRemoteLogAuxState(TopicPartition topicPartition, - Integer currentLeaderEpoch, - Long leaderLocalLogStartOffset, - Integer epochForLeaderLocalLogStartOffset, - Long leaderLogStartOffset) throws IOException, RemoteStorageException { - - UnifiedLog unifiedLog = replicaMgr.localLogOrException(topicPartition); - - long nextOffset; - - if (unifiedLog.remoteStorageSystemEnable() && unifiedLog.config().remoteStorageEnable()) { - if (replicaMgr.remoteLogManager().isEmpty()) throw new IllegalStateException("RemoteLogManager is not yet instantiated"); - - RemoteLogManager rlm = replicaMgr.remoteLogManager().get(); - - // Find the respective leader epoch for (leaderLocalLogStartOffset - 1). We need to build the leader epoch cache - // until that offset - long previousOffsetToLeaderLocalLogStartOffset = leaderLocalLogStartOffset - 1; - int targetEpoch; - // If the existing epoch is 0, no need to fetch from earlier epoch as the desired offset(leaderLogStartOffset - 1) - // will have the same epoch. - if (epochForLeaderLocalLogStartOffset == 0) { - targetEpoch = epochForLeaderLocalLogStartOffset; - } else { - // Fetch the earlier epoch/end-offset(exclusive) from the leader. - EpochEndOffset earlierEpochEndOffset = fetchEarlierEpochEndOffset(epochForLeaderLocalLogStartOffset, topicPartition, currentLeaderEpoch); - // Check if the target offset lies within the range of earlier epoch. Here, epoch's end-offset is exclusive. - if (earlierEpochEndOffset.endOffset() > previousOffsetToLeaderLocalLogStartOffset) { - // Always use the leader epoch from returned earlierEpochEndOffset. - // This gives the respective leader epoch, that will handle any gaps in epochs. - // For ex, leader epoch cache contains: - // leader-epoch start-offset - // 0 20 - // 1 85 - // <2> - gap no messages were appended in this leader epoch. - // 3 90 - // 4 98 - // There is a gap in leader epoch. For leaderLocalLogStartOffset as 90, leader-epoch is 3. - // fetchEarlierEpochEndOffset(2) will return leader-epoch as 1, end-offset as 90. - // So, for offset 89, we should return leader epoch as 1 like below. - targetEpoch = earlierEpochEndOffset.leaderEpoch(); - } else { - targetEpoch = epochForLeaderLocalLogStartOffset; - } - } - - Optional maybeRlsm = rlm.fetchRemoteLogSegmentMetadata(topicPartition, targetEpoch, previousOffsetToLeaderLocalLogStartOffset); - - if (maybeRlsm.isPresent()) { - RemoteLogSegmentMetadata remoteLogSegmentMetadata = maybeRlsm.get(); - // Build leader epoch cache, producer snapshots until remoteLogSegmentMetadata.endOffset() and start - // segments from (remoteLogSegmentMetadata.endOffset() + 1) - // Assign nextOffset with the offset from which next fetch should happen. - nextOffset = remoteLogSegmentMetadata.endOffset() + 1; - - // Truncate the existing local log before restoring the leader epoch cache and producer snapshots. - Partition partition = replicaMgr.getPartitionOrException(topicPartition); - partition.truncateFullyAndStartAt(nextOffset, false, Option.apply(leaderLogStartOffset)); - - // Increment start offsets - unifiedLog.maybeIncrementLogStartOffset(leaderLogStartOffset, LeaderOffsetIncremented); - unifiedLog.maybeIncrementLocalLogStartOffset(nextOffset, LeaderOffsetIncremented); - - // Build leader epoch cache. - List epochs = readLeaderEpochCheckpoint(rlm, remoteLogSegmentMetadata); - if (unifiedLog.leaderEpochCache().isDefined()) { - unifiedLog.leaderEpochCache().get().assign(epochs); - } - - log.debug("Updated the epoch cache from remote tier till offset: {} with size: {} for {}", leaderLocalLogStartOffset, epochs.size(), partition); - - // Restore producer snapshot - File snapshotFile = LogFileUtils.producerSnapshotFile(unifiedLog.dir(), nextOffset); - buildProducerSnapshotFile(snapshotFile, remoteLogSegmentMetadata, rlm); - - // Reload producer snapshots. - unifiedLog.producerStateManager().truncateFullyAndReloadSnapshots(); - unifiedLog.loadProducerState(nextOffset); - log.debug("Built the leader epoch cache and producer snapshots from remote tier for {}, " + - "with active producers size: {}, leaderLogStartOffset: {}, and logEndOffset: {}", - partition, unifiedLog.producerStateManager().activeProducers().size(), leaderLogStartOffset, nextOffset); - } else { - throw new RemoteStorageException("Couldn't build the state from remote store for partition: " + topicPartition + - ", currentLeaderEpoch: " + currentLeaderEpoch + - ", leaderLocalLogStartOffset: " + leaderLocalLogStartOffset + - ", leaderLogStartOffset: " + leaderLogStartOffset + - ", epoch: " + targetEpoch + - "as the previous remote log segment metadata was not found"); - } - } else { - // If the tiered storage is not enabled throw an exception back so that it will retry until the tiered storage - // is set as expected. - throw new RemoteStorageException("Couldn't build the state from remote store for partition " + topicPartition + ", as remote log storage is not yet enabled"); - } - - return nextOffset; - } -} diff --git a/core/src/main/java/kafka/server/TierStateMachine.java b/core/src/main/java/kafka/server/TierStateMachine.java index 58a44cc647232..085e6c025dd4c 100644 --- a/core/src/main/java/kafka/server/TierStateMachine.java +++ b/core/src/main/java/kafka/server/TierStateMachine.java @@ -17,15 +17,65 @@ package kafka.server; -import java.util.Optional; +import java.io.BufferedReader; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import kafka.cluster.Partition; +import kafka.log.UnifiedLog; +import kafka.log.remote.RemoteLogManager; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.FetchResponseData.PartitionData; +import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData; +import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.common.CheckpointFile; +import org.apache.kafka.server.common.OffsetAndEpoch; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteStorageException; +import org.apache.kafka.server.log.remote.storage.RemoteStorageManager; +import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile; +import org.apache.kafka.storage.internals.log.EpochEntry; +import org.apache.kafka.storage.internals.log.LogFileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; +import scala.collection.JavaConverters; + +import static org.apache.kafka.storage.internals.log.LogStartOffsetIncrementReason.LeaderOffsetIncremented; /** - * This interface defines the APIs needed to handle any state transitions related to tiering + * This class defines the APIs and implementation needed to handle any state transitions related to tiering + * + * When started, the tier state machine will fetch the local log start offset of the + * leader and then build the follower's remote log aux state until the leader's + * local log start offset. */ -public interface TierStateMachine { +public class TierStateMachine { + private static final Logger log = LoggerFactory.getLogger(TierStateMachine.class); + + private final LeaderEndPoint leader; + private final ReplicaManager replicaMgr; + private final boolean useFutureLog; + public TierStateMachine(LeaderEndPoint leader, + ReplicaManager replicaMgr, + boolean useFutureLog) { + this.leader = leader; + this.replicaMgr = replicaMgr; + this.useFutureLog = useFutureLog; + } /** * Start the tier state machine for the provided topic partition. @@ -40,19 +90,176 @@ public interface TierStateMachine { */ PartitionFetchState start(TopicPartition topicPartition, PartitionFetchState currentFetchState, - PartitionData fetchPartitionData) throws Exception; + PartitionData fetchPartitionData) throws Exception { + OffsetAndEpoch epochAndLeaderLocalStartOffset = leader.fetchEarliestLocalOffset(topicPartition, currentFetchState.currentLeaderEpoch()); + int epoch = epochAndLeaderLocalStartOffset.leaderEpoch(); + long leaderLocalStartOffset = epochAndLeaderLocalStartOffset.offset(); + + long offsetToFetch; + replicaMgr.brokerTopicStats().topicStats(topicPartition.topic()).buildRemoteLogAuxStateRequestRate().mark(); + replicaMgr.brokerTopicStats().allTopicsStats().buildRemoteLogAuxStateRequestRate().mark(); + + UnifiedLog unifiedLog; + if (useFutureLog) { + unifiedLog = replicaMgr.futureLogOrException(topicPartition); + } else { + unifiedLog = replicaMgr.localLogOrException(topicPartition); + } + + try { + offsetToFetch = buildRemoteLogAuxState(topicPartition, currentFetchState.currentLeaderEpoch(), leaderLocalStartOffset, epoch, fetchPartitionData.logStartOffset(), unifiedLog); + } catch (RemoteStorageException e) { + replicaMgr.brokerTopicStats().topicStats(topicPartition.topic()).failedBuildRemoteLogAuxStateRate().mark(); + replicaMgr.brokerTopicStats().allTopicsStats().failedBuildRemoteLogAuxStateRate().mark(); + throw e; + } + + OffsetAndEpoch fetchLatestOffsetResult = leader.fetchLatestOffset(topicPartition, currentFetchState.currentLeaderEpoch()); + long leaderEndOffset = fetchLatestOffsetResult.offset(); + + long initialLag = leaderEndOffset - offsetToFetch; + + return PartitionFetchState.apply(currentFetchState.topicId(), offsetToFetch, Option.apply(initialLag), currentFetchState.currentLeaderEpoch(), + Fetching$.MODULE$, unifiedLog.latestEpoch()); + + } + + private OffsetForLeaderEpochResponseData.EpochEndOffset fetchEarlierEpochEndOffset(Integer epoch, + TopicPartition partition, + Integer currentLeaderEpoch) { + int previousEpoch = epoch - 1; + + // Find the end-offset for the epoch earlier to the given epoch from the leader + Map partitionsWithEpochs = new HashMap<>(); + partitionsWithEpochs.put(partition, new OffsetForLeaderEpochRequestData.OffsetForLeaderPartition().setPartition(partition.partition()).setCurrentLeaderEpoch(currentLeaderEpoch).setLeaderEpoch(previousEpoch)); + Option maybeEpochEndOffset = leader.fetchEpochEndOffsets(JavaConverters.mapAsScalaMap(partitionsWithEpochs)).get(partition); + if (maybeEpochEndOffset.isEmpty()) { + throw new KafkaException("No response received for partition: " + partition); + } + + OffsetForLeaderEpochResponseData.EpochEndOffset epochEndOffset = maybeEpochEndOffset.get(); + if (epochEndOffset.errorCode() != Errors.NONE.code()) { + throw Errors.forCode(epochEndOffset.errorCode()).exception(); + } + + return epochEndOffset; + } + + private List readLeaderEpochCheckpoint(RemoteLogManager rlm, + RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws IOException, RemoteStorageException { + InputStream inputStream = rlm.storageManager().fetchIndex(remoteLogSegmentMetadata, RemoteStorageManager.IndexType.LEADER_EPOCH); + try (BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))) { + CheckpointFile.CheckpointReadBuffer readBuffer = new CheckpointFile.CheckpointReadBuffer<>("", bufferedReader, 0, LeaderEpochCheckpointFile.FORMATTER); + return readBuffer.read(); + } + } + + private void buildProducerSnapshotFile(UnifiedLog unifiedLog, + long nextOffset, + RemoteLogSegmentMetadata remoteLogSegmentMetadata, + RemoteLogManager rlm) throws IOException, RemoteStorageException { + // Restore producer snapshot + File snapshotFile = LogFileUtils.producerSnapshotFile(unifiedLog.dir(), nextOffset); + Path tmpSnapshotFile = Paths.get(snapshotFile.getAbsolutePath() + ".tmp"); + // Copy it to snapshot file in atomic manner. + Files.copy(rlm.storageManager().fetchIndex(remoteLogSegmentMetadata, RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT), + tmpSnapshotFile, StandardCopyOption.REPLACE_EXISTING); + Utils.atomicMoveWithFallback(tmpSnapshotFile, snapshotFile.toPath(), false); + + // Reload producer snapshots. + unifiedLog.producerStateManager().truncateFullyAndReloadSnapshots(); + unifiedLog.loadProducerState(nextOffset); + } /** - * Optionally advance the state of the tier state machine, based on the - * current PartitionFetchState. The decision to advance the tier - * state machine is implementation specific. - * - * @param topicPartition the topic partition - * @param currentFetchState the current PartitionFetchState which will - * be used to derive the return value - * - * @return the new PartitionFetchState if the tier state machine was advanced, otherwise, return the currentFetchState + * It tries to build the required state for this partition from leader and remote storage so that it can start + * fetching records from the leader. The return value is the next offset to fetch from the leader, which is the + * next offset following the end offset of the remote log portion. */ - Optional maybeAdvanceState(TopicPartition topicPartition, - PartitionFetchState currentFetchState); + private Long buildRemoteLogAuxState(TopicPartition topicPartition, + Integer currentLeaderEpoch, + Long leaderLocalLogStartOffset, + Integer epochForLeaderLocalLogStartOffset, + Long leaderLogStartOffset, + UnifiedLog unifiedLog) throws IOException, RemoteStorageException { + + if (!unifiedLog.remoteStorageSystemEnable() || !unifiedLog.config().remoteStorageEnable()) { + // If the tiered storage is not enabled throw an exception back so that it will retry until the tiered storage + // is set as expected. + throw new RemoteStorageException("Couldn't build the state from remote store for partition " + topicPartition + ", as remote log storage is not yet enabled"); + } + + if (replicaMgr.remoteLogManager().isEmpty()) + throw new IllegalStateException("RemoteLogManager is not yet instantiated"); + + RemoteLogManager rlm = replicaMgr.remoteLogManager().get(); + + // Find the respective leader epoch for (leaderLocalLogStartOffset - 1). We need to build the leader epoch cache + // until that offset + long previousOffsetToLeaderLocalLogStartOffset = leaderLocalLogStartOffset - 1; + int targetEpoch; + // If the existing epoch is 0, no need to fetch from earlier epoch as the desired offset(leaderLogStartOffset - 1) + // will have the same epoch. + if (epochForLeaderLocalLogStartOffset == 0) { + targetEpoch = epochForLeaderLocalLogStartOffset; + } else { + // Fetch the earlier epoch/end-offset(exclusive) from the leader. + OffsetForLeaderEpochResponseData.EpochEndOffset earlierEpochEndOffset = fetchEarlierEpochEndOffset(epochForLeaderLocalLogStartOffset, topicPartition, currentLeaderEpoch); + // Check if the target offset lies within the range of earlier epoch. Here, epoch's end-offset is exclusive. + if (earlierEpochEndOffset.endOffset() > previousOffsetToLeaderLocalLogStartOffset) { + // Always use the leader epoch from returned earlierEpochEndOffset. + // This gives the respective leader epoch, that will handle any gaps in epochs. + // For ex, leader epoch cache contains: + // leader-epoch start-offset + // 0 20 + // 1 85 + // <2> - gap no messages were appended in this leader epoch. + // 3 90 + // 4 98 + // There is a gap in leader epoch. For leaderLocalLogStartOffset as 90, leader-epoch is 3. + // fetchEarlierEpochEndOffset(2) will return leader-epoch as 1, end-offset as 90. + // So, for offset 89, we should return leader epoch as 1 like below. + targetEpoch = earlierEpochEndOffset.leaderEpoch(); + } else { + targetEpoch = epochForLeaderLocalLogStartOffset; + } + } + + RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlm.fetchRemoteLogSegmentMetadata(topicPartition, targetEpoch, previousOffsetToLeaderLocalLogStartOffset) + .orElseThrow(() -> new RemoteStorageException("Couldn't build the state from remote store for partition: " + topicPartition + + ", currentLeaderEpoch: " + currentLeaderEpoch + + ", leaderLocalLogStartOffset: " + leaderLocalLogStartOffset + + ", leaderLogStartOffset: " + leaderLogStartOffset + + ", epoch: " + targetEpoch + + "as the previous remote log segment metadata was not found")); + + + // Build leader epoch cache, producer snapshots until remoteLogSegmentMetadata.endOffset() and start + // segments from (remoteLogSegmentMetadata.endOffset() + 1) + // Assign nextOffset with the offset from which next fetch should happen. + long nextOffset = remoteLogSegmentMetadata.endOffset() + 1; + + // Truncate the existing local log before restoring the leader epoch cache and producer snapshots. + Partition partition = replicaMgr.getPartitionOrException(topicPartition); + partition.truncateFullyAndStartAt(nextOffset, useFutureLog, Option.apply(leaderLogStartOffset)); + // Increment start offsets + unifiedLog.maybeIncrementLogStartOffset(leaderLogStartOffset, LeaderOffsetIncremented); + unifiedLog.maybeIncrementLocalLogStartOffset(nextOffset, LeaderOffsetIncremented); + + // Build leader epoch cache. + List epochs = readLeaderEpochCheckpoint(rlm, remoteLogSegmentMetadata); + if (unifiedLog.leaderEpochCache().isDefined()) { + unifiedLog.leaderEpochCache().get().assign(epochs); + } + + log.info("Updated the epoch cache from remote tier till offset: {} with size: {} for {}", leaderLocalLogStartOffset, epochs.size(), partition); + + buildProducerSnapshotFile(unifiedLog, nextOffset, remoteLogSegmentMetadata, rlm); + + log.debug("Built the leader epoch cache and producer snapshots from remote tier for {}, " + + "with active producers size: {}, leaderLogStartOffset: {}, and logEndOffset: {}", + partition, unifiedLog.producerStateManager().activeProducers().size(), leaderLogStartOffset, nextOffset); + + return nextOffset; + } } diff --git a/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java b/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java index 9182f9c4bcd1f..1d422461678f5 100644 --- a/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java +++ b/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java @@ -179,7 +179,7 @@ public KafkaApis build() { if (metrics == null) throw new RuntimeException("You must set metrics"); if (quotas == null) throw new RuntimeException("You must set quotas"); if (fetchManager == null) throw new RuntimeException("You must set fetchManager"); - if (brokerTopicStats == null) brokerTopicStats = new BrokerTopicStats(Optional.of(config)); + if (brokerTopicStats == null) brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig().enableRemoteStorageSystem()); if (apiVersionManager == null) throw new RuntimeException("You must set apiVersionManager"); return new KafkaApis(requestChannel, diff --git a/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java b/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java index 13b60a539f9ef..5e8cf2dcdc64c 100644 --- a/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java +++ b/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java @@ -185,7 +185,7 @@ public ReplicaManager build() { if (metadataCache == null) throw new RuntimeException("You must set metadataCache"); if (logDirFailureChannel == null) throw new RuntimeException("You must set logDirFailureChannel"); if (alterPartitionManager == null) throw new RuntimeException("You must set alterIsrManager"); - if (brokerTopicStats == null) brokerTopicStats = new BrokerTopicStats(Optional.of(config)); + if (brokerTopicStats == null) brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig().enableRemoteStorageSystem()); // Initialize metrics in the end just before passing it to ReplicaManager to ensure ReplicaManager closes the // metrics correctly. There might be a resource leak if it is initialized and an exception occurs between // its initialization and creation of ReplicaManager. diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 7421aed637d69..04dce7a025556 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -19,7 +19,7 @@ package kafka.admin import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit -import java.util.{Collections, Properties} +import java.util.{Collections, Optional, Properties} import joptsimple._ import kafka.server.{DynamicBrokerConfig, DynamicConfig, KafkaConfig} import kafka.utils.Implicits._ @@ -210,15 +210,19 @@ object ConfigCommand extends Logging { } } - def createPasswordEncoder(encoderConfigs: Map[String, String]): PasswordEncoder = { - encoderConfigs.get(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG) - val encoderSecret = encoderConfigs.getOrElse(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, - throw new IllegalArgumentException("Password encoder secret not specified")) + def createPasswordEncoder(encoderConfigs: java.util.Map[String, String]): PasswordEncoder = { + val encoderSecret = Optional.ofNullable(encoderConfigs.get(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG)) + .orElseThrow(() => new IllegalArgumentException("Password encoder secret not specified")) PasswordEncoder.encrypting(new Password(encoderSecret), null, - encoderConfigs.getOrElse(PasswordEncoderConfigs.PASSWORD_ENCODER_CIPHER_ALGORITHM_CONFIG, PasswordEncoderConfigs.PASSWORD_ENCODER_CIPHER_ALGORITHM_DEFAULT), - encoderConfigs.get(PasswordEncoderConfigs.PASSWORD_ENCODER_KEY_LENGTH_CONFIG).map(_.toInt).getOrElse(PasswordEncoderConfigs.PASSWORD_ENCODER_KEY_LENGTH_DEFAULT), - encoderConfigs.get(PasswordEncoderConfigs.PASSWORD_ENCODER_ITERATIONS_CONFIG).map(_.toInt).getOrElse(PasswordEncoderConfigs.PASSWORD_ENCODER_ITERATIONS_DEFAULT)) + encoderConfigs.getOrDefault(PasswordEncoderConfigs.PASSWORD_ENCODER_CIPHER_ALGORITHM_CONFIG, PasswordEncoderConfigs.PASSWORD_ENCODER_CIPHER_ALGORITHM_DEFAULT), + Optional.ofNullable(encoderConfigs.get(PasswordEncoderConfigs.PASSWORD_ENCODER_KEY_LENGTH_CONFIG)) + .map[Int](Integer.parseInt) + .orElse(PasswordEncoderConfigs.PASSWORD_ENCODER_KEY_LENGTH_DEFAULT), + Optional.ofNullable(encoderConfigs.get(PasswordEncoderConfigs.PASSWORD_ENCODER_ITERATIONS_CONFIG)) + .map[Int](Integer.parseInt) + .orElse(PasswordEncoderConfigs.PASSWORD_ENCODER_ITERATIONS_DEFAULT) + ) } /** @@ -244,8 +248,11 @@ object ConfigCommand extends Logging { " to override the default encoding parameters. Password encoder configs will not be persisted" + " in ZooKeeper." ) - - val passwordEncoder = createPasswordEncoder(passwordEncoderConfigs.asScala) + val passwordConfigsMap = new java.util.HashMap[String, String] + passwordEncoderConfigs.forEach { (key, value) => + passwordConfigsMap.put(key.toString, value.toString) + } + val passwordEncoder = createPasswordEncoder(passwordConfigsMap) passwordConfigs.foreach { configName => val encodedValue = passwordEncoder.encode(new Password(configsToBeAdded.getProperty(configName))) configsToBeAdded.setProperty(configName, encodedValue) diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index ede63cd3c0a60..e5835201fa356 100755 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -43,6 +43,10 @@ object Broker { new Broker(id, endPoints, rack, emptySupportedFeatures) } + def apply(id: Int, endPoint: EndPoint, rack: Option[String]): Broker = { + new Broker(id, Seq(endPoint), rack, emptySupportedFeatures) + } + private def supportedFeatures(features: java.util.Map[String, VersionRange]): java.util .Map[String, SupportedVersionRange] = { features.asScala.map { case (name, range) => diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 980c6ccb258fe..e6783b7d190c1 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -427,7 +427,7 @@ class Partition(val topicPartition: TopicPartition, * @param highWatermarkCheckpoints Checkpoint to load initial high watermark from * @return true iff the future replica is created */ - def maybeCreateFutureReplica(logDir: String, highWatermarkCheckpoints: OffsetCheckpoints): Boolean = { + def maybeCreateFutureReplica(logDir: String, highWatermarkCheckpoints: OffsetCheckpoints, topicId: Option[Uuid] = topicId): Boolean = { // The writeLock is needed to make sure that while the caller checks the log directory of the // current replica and the existence of the future replica, no other thread can update the log directory of the // current replica or remove the future replica. diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index a2d6853ddc5de..e89876e443ce3 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -494,9 +494,9 @@ class GroupMetadataManager(brokerId: Int, if (isTxnOffsetCommit) { addProducerGroup(producerId, group.groupId) - group.prepareTxnOffsetCommit(producerId, offsetMetadata) + group.prepareTxnOffsetCommit(producerId, filteredOffsetMetadata) } else { - group.prepareOffsetCommit(offsetMetadata) + group.prepareOffsetCommit(filteredOffsetMetadata) } appendForGroup(group, records, requestLocal, putCacheCallback, verificationGuards) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index c4782dd95cea3..94445bed2c96b 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -36,7 +36,7 @@ import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} -import org.apache.kafka.server.config.Defaults +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.util.Scheduler import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation} @@ -812,7 +812,7 @@ private[transaction] case class TransactionConfig(transactionalIdExpirationMs: I transactionLogMinInsyncReplicas: Int = TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_DEFAULT, abortTimedOutTransactionsIntervalMs: Int = TransactionStateManagerConfigs.TRANSACTIONS_ABORT_TIMED_OUT_TRANSACTION_CLEANUP_INTERVAL_MS_DEFAULT, removeExpiredTransactionalIdsIntervalMs: Int = TransactionStateManagerConfigs.TRANSACTIONS_REMOVE_EXPIRED_TRANSACTIONAL_ID_CLEANUP_INTERVAL_MS_DEFAULT, - requestTimeoutMs: Int = Defaults.REQUEST_TIMEOUT_MS) + requestTimeoutMs: Int = ServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT) case class TransactionalIdAndProducerIdEpoch(transactionalId: String, producerId: Long, producerEpoch: Short) { override def toString: String = { diff --git a/core/src/main/scala/kafka/docker/KafkaDockerWrapper.scala b/core/src/main/scala/kafka/docker/KafkaDockerWrapper.scala index eadbdb8aefc53..8db498db12738 100644 --- a/core/src/main/scala/kafka/docker/KafkaDockerWrapper.scala +++ b/core/src/main/scala/kafka/docker/KafkaDockerWrapper.scala @@ -18,7 +18,7 @@ package kafka.docker import kafka.Kafka import kafka.tools.StorageTool -import kafka.utils.Exit +import kafka.utils.{Exit, Logging} import net.sourceforge.argparse4j.ArgumentParsers import net.sourceforge.argparse4j.impl.Arguments.store import net.sourceforge.argparse4j.inf.Namespace @@ -26,7 +26,7 @@ import net.sourceforge.argparse4j.inf.Namespace import java.nio.charset.StandardCharsets import java.nio.file.{Files, Path, Paths, StandardCopyOption, StandardOpenOption} -object KafkaDockerWrapper { +object KafkaDockerWrapper extends Logging { def main(args: Array[String]): Unit = { val namespace = parseArguments(args) val command = namespace.getString("command") @@ -48,6 +48,7 @@ object KafkaDockerWrapper { StorageTool.main(formatCmd) case "start" => val configFile = namespace.getString("config") + info("Starting Kafka server in the native mode.") Kafka.main(Array(configFile)) case _ => throw new RuntimeException(s"Unknown operation $command. " + diff --git a/core/src/main/scala/kafka/log/LocalLog.scala b/core/src/main/scala/kafka/log/LocalLog.scala index b2121f5312d7b..09fa464af043c 100644 --- a/core/src/main/scala/kafka/log/LocalLog.scala +++ b/core/src/main/scala/kafka/log/LocalLog.scala @@ -383,12 +383,19 @@ class LocalLog(@volatile private var _dir: File, val segment = segmentOpt.get val baseOffset = segment.baseOffset - val maxPosition = - // Use the max offset position if it is on this segment; otherwise, the segment size is the limit. - if (maxOffsetMetadata.segmentBaseOffset == segment.baseOffset) maxOffsetMetadata.relativePositionInSegment - else segment.size - - fetchDataInfo = segment.read(startOffset, maxLength, maxPosition, minOneMessage) + // 1. If `maxOffsetMetadata#segmentBaseOffset < segment#baseOffset`, then return maxPosition as empty. + // 2. Use the max-offset position if it is on this segment; otherwise, the segment size is the limit. + // 3. When maxOffsetMetadata is message-offset-only, then we don't know the relativePositionInSegment so + // return maxPosition as empty to avoid reading beyond the max-offset + val maxPositionOpt: Optional[java.lang.Long] = + if (segment.baseOffset < maxOffsetMetadata.segmentBaseOffset) + Optional.of(segment.size) + else if (segment.baseOffset == maxOffsetMetadata.segmentBaseOffset && !maxOffsetMetadata.messageOffsetOnly()) + Optional.of(maxOffsetMetadata.relativePositionInSegment) + else + Optional.empty() + + fetchDataInfo = segment.read(startOffset, maxLength, maxPositionOpt, minOneMessage) if (fetchDataInfo != null) { if (includeAbortedTxns) fetchDataInfo = addAbortedTransactions(startOffset, segment, fetchDataInfo) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 66afce2a1e542..1265e979373cd 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -32,6 +32,7 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{BufferSupplier, Time} +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.ShutdownableThread import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, LastRecord, LogDirFailureChannel, LogSegment, LogSegmentOffsetOverflowException, OffsetMap, SkimpyOffsetMap, TransactionIndex} @@ -160,11 +161,18 @@ class LogCleaner(initialConfig: CleanerConfig, /** * Stop the background cleaner threads */ - def shutdown(): Unit = { + private[this] def shutdownCleaners(): Unit = { info("Shutting down the log cleaner.") + cleaners.foreach(_.shutdown()) + cleaners.clear() + } + + /** + * Stop the background cleaner threads + */ + def shutdown(): Unit = { try { - cleaners.foreach(_.shutdown()) - cleaners.clear() + shutdownCleaners() } finally { removeMetrics() } @@ -219,8 +227,8 @@ class LogCleaner(initialConfig: CleanerConfig, info(s"Updating logCleanerIoMaxBytesPerSecond: $maxIoBytesPerSecond") throttler.updateDesiredRatePerSec(maxIoBytesPerSecond) } - - shutdown() + // call shutdownCleaners() instead of shutdown to avoid unnecessary deletion of metrics + shutdownCleaners() startup() } @@ -499,7 +507,7 @@ object LogCleaner { CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, - KafkaConfig.MessageMaxBytesProp, + ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP ) diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 3bc6533117cba..d7599e569ab25 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -1562,7 +1562,7 @@ object LogManager { keepPartitionMetadataFile: Boolean): LogManager = { val defaultProps = config.extractLogConfigMap - LogConfig.validateBrokerLogConfigValues(defaultProps, config.isRemoteLogStorageSystemEnabled) + LogConfig.validateBrokerLogConfigValues(defaultProps, config.remoteLogManagerConfig.enableRemoteStorageSystem()) val defaultLogConfig = new LogConfig(defaultProps) val cleanerConfig = LogCleaner.cleanerConfig(config) diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 8302123fd1fb5..60adc0be86616 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -361,7 +361,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, val offsetMetadata = highWatermarkMetadata if (offsetMetadata.messageOffsetOnly) { lock.synchronized { - val fullOffset = convertToOffsetMetadataOrThrow(highWatermark) + val fullOffset = maybeConvertToOffsetMetadata(highWatermark) updateHighWatermarkMetadata(fullOffset) fullOffset } @@ -405,7 +405,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, case Some(offsetMetadata) if offsetMetadata.messageOffset < highWatermarkMetadata.messageOffset => if (offsetMetadata.messageOffsetOnly) { lock synchronized { - val fullOffset = convertToOffsetMetadataOrThrow(offsetMetadata.messageOffset) + val fullOffset = maybeConvertToOffsetMetadata(offsetMetadata.messageOffset) if (firstUnstableOffsetMetadata.contains(offsetMetadata)) firstUnstableOffsetMetadata = Some(fullOffset) fullOffset @@ -965,7 +965,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, val updatedFirstUnstableOffset = producerStateManager.firstUnstableOffset.asScala match { case Some(logOffsetMetadata) if logOffsetMetadata.messageOffsetOnly || logOffsetMetadata.messageOffset < logStartOffset => val offset = math.max(logOffsetMetadata.messageOffset, logStartOffset) - Some(convertToOffsetMetadataOrThrow(offset)) + Some(maybeConvertToOffsetMetadata(offset)) case other => other } @@ -1426,11 +1426,18 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Given a message offset, find its corresponding offset metadata in the log. - * If the message offset is out of range, throw an OffsetOutOfRangeException + * 1. If the message offset is less than the log-start-offset (or) local-log-start-offset, then it returns the + * message-only metadata. + * 2. If the message offset is beyond the log-end-offset, then it returns the message-only metadata. + * 3. For all other cases, it returns the offset metadata from the log. */ - private def convertToOffsetMetadataOrThrow(offset: Long): LogOffsetMetadata = { - checkLogStartOffset(offset) - localLog.convertToOffsetMetadataOrThrow(offset) + private[log] def maybeConvertToOffsetMetadata(offset: Long): LogOffsetMetadata = { + try { + localLog.convertToOffsetMetadataOrThrow(offset) + } catch { + case _: OffsetOutOfRangeException => + new LogOffsetMetadata(offset) + } } /** @@ -1912,7 +1919,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, * to ensure no other logcleaner threads and retention thread can work on the same segment. */ private[log] def getFirstBatchTimestampForSegments(segments: util.Collection[LogSegment]): util.Collection[java.lang.Long] = { - LogSegments.getFirstBatchTimestampForSegments(segments) + segments.stream().map[java.lang.Long](s => s.getFirstBatchTimestamp).collect(Collectors.toList()) } /** diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala index e2064602c81a3..83973ede23dbf 100755 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala @@ -20,10 +20,12 @@ package kafka.metrics -import kafka.utils.{CoreUtils, VerifiableProperties} +import kafka.utils.VerifiableProperties import org.apache.kafka.server.metrics.MetricConfigs +import org.apache.kafka.server.util.Csv import scala.collection.Seq +import scala.jdk.CollectionConverters._ class KafkaMetricsConfig(props: VerifiableProperties) { @@ -31,8 +33,8 @@ class KafkaMetricsConfig(props: VerifiableProperties) { * Comma-separated list of reporter types. These classes should be on the * classpath and will be instantiated at run-time. */ - val reporters: Seq[String] = CoreUtils.parseCsvList(props.getString(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG, - MetricConfigs.KAFKA_METRIC_REPORTER_CLASSES_DEFAULT)) + val reporters: Seq[String] = Csv.parseCsvList(props.getString(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG, + MetricConfigs.KAFKA_METRIC_REPORTER_CLASSES_DEFAULT)).asScala /** * The metrics polling interval (in seconds). diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index 54986f52c85a3..0900b94ef9f4f 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -95,6 +95,10 @@ object RequestConvertToJson { case req: RenewDelegationTokenRequest => RenewDelegationTokenRequestDataJsonConverter.write(req.data, request.version) case req: SaslAuthenticateRequest => SaslAuthenticateRequestDataJsonConverter.write(req.data, request.version) case req: SaslHandshakeRequest => SaslHandshakeRequestDataJsonConverter.write(req.data, request.version) + case req: ShareAcknowledgeRequest => ShareAcknowledgeRequestDataJsonConverter.write(req.data, request.version) + case req: ShareFetchRequest => ShareFetchRequestDataJsonConverter.write(req.data, request.version) + case req: ShareGroupDescribeRequest => ShareGroupDescribeRequestDataJsonConverter.write(req.data, request.version) + case req: ShareGroupHeartbeatRequest => ShareGroupHeartbeatRequestDataJsonConverter.write(req.data, request.version) case req: StopReplicaRequest => StopReplicaRequestDataJsonConverter.write(req.data, request.version) case req: SyncGroupRequest => SyncGroupRequestDataJsonConverter.write(req.data, request.version) case req: TxnOffsetCommitRequest => TxnOffsetCommitRequestDataJsonConverter.write(req.data, request.version) @@ -178,6 +182,10 @@ object RequestConvertToJson { case res: RenewDelegationTokenResponse => RenewDelegationTokenResponseDataJsonConverter.write(res.data, version) case res: SaslAuthenticateResponse => SaslAuthenticateResponseDataJsonConverter.write(res.data, version) case res: SaslHandshakeResponse => SaslHandshakeResponseDataJsonConverter.write(res.data, version) + case res: ShareAcknowledgeResponse => ShareAcknowledgeResponseDataJsonConverter.write(res.data, version) + case res: ShareFetchResponse => ShareFetchResponseDataJsonConverter.write(res.data, version) + case res: ShareGroupDescribeResponse => ShareGroupDescribeResponseDataJsonConverter.write(res.data, version) + case res: ShareGroupHeartbeatResponse => ShareGroupHeartbeatResponseDataJsonConverter.write(res.data, version) case res: StopReplicaResponse => StopReplicaResponseDataJsonConverter.write(res.data, version) case res: SyncGroupResponse => SyncGroupResponseDataJsonConverter.write(res.data, version) case res: TxnOffsetCommitResponse => TxnOffsetCommitResponseDataJsonConverter.write(res.data, version) diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 6dd4bc5051328..42d1e6585e6cf 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -47,7 +47,7 @@ import org.apache.kafka.common.utils.{KafkaThread, LogContext, Time, Utils} import org.apache.kafka.common.{Endpoint, KafkaException, MetricName, Reconfigurable} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.CredentialProvider -import org.apache.kafka.server.config.QuotaConfigs +import org.apache.kafka.server.config.{ServerConfigs, QuotaConfigs} import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.FutureUtils import org.slf4j.event.Level @@ -425,7 +425,7 @@ object SocketServer { object DataPlaneAcceptor { val ThreadPrefix: String = "data-plane" val MetricPrefix: String = "" - val ListenerReconfigurableConfigs: Set[String] = Set(KafkaConfig.NumNetworkThreadsProp) + val ListenerReconfigurableConfigs: Set[String] = Set(ServerConfigs.NUM_NETWORK_THREADS_CONFIG) } class DataPlaneAcceptor(socketServer: SocketServer, @@ -506,7 +506,7 @@ class DataPlaneAcceptor(socketServer: SocketServer, * the configs have passed validation using [[validateReconfiguration( Map )]]. */ override def reconfigure(configs: util.Map[String, _]): Unit = { - val newNumNetworkThreads = configs.get(KafkaConfig.NumNetworkThreadsProp).asInstanceOf[Int] + val newNumNetworkThreads = configs.get(ServerConfigs.NUM_NETWORK_THREADS_CONFIG).asInstanceOf[Int] if (newNumNetworkThreads != processors.length) { info(s"Resizing network thread pool size for ${endPoint.listenerName} listener from ${processors.length} to $newNumNetworkThreads") @@ -522,7 +522,7 @@ class DataPlaneAcceptor(socketServer: SocketServer, * Configure this class with the given key-value pairs */ override def configure(configs: util.Map[String, _]): Unit = { - addProcessors(configs.get(KafkaConfig.NumNetworkThreadsProp).asInstanceOf[Int]) + addProcessors(configs.get(ServerConfigs.NUM_NETWORK_THREADS_CONFIG).asInstanceOf[Int]) } } diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 15f87b9c247e9..5e107aa148772 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -223,7 +223,7 @@ final class KafkaMetadataLog private ( override def highWatermark: LogOffsetMetadata = { val hwm = log.fetchOffsetSnapshot.highWatermark - val segmentPosition: Optional[OffsetMetadata] = if (hwm.messageOffsetOnly) { + val segmentPosition: Optional[OffsetMetadata] = if (!hwm.messageOffsetOnly) { Optional.of(SegmentPosition(hwm.segmentBaseOffset, hwm.relativePositionInSegment)) } else { Optional.empty() diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index f44197abd38c7..6bf8bd893ba70 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -30,9 +30,7 @@ import kafka.utils.CoreUtils import kafka.utils.FileLock import kafka.utils.Logging import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, NetworkClient} -import org.apache.kafka.common.KafkaException -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.Uuid +import org.apache.kafka.common.{KafkaException, Node, TopicPartition, Uuid} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ChannelBuilders, ListenerName, NetworkReceive, Selectable, Selector} @@ -41,7 +39,7 @@ import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{LogContext, Time, Utils} -import org.apache.kafka.raft.{FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, RaftClient, QuorumConfig, ReplicatedLog} +import org.apache.kafka.raft.{FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, QuorumConfig, RaftClient, ReplicatedLog} import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.server.util.KafkaScheduler @@ -49,6 +47,7 @@ import org.apache.kafka.server.fault.FaultHandler import org.apache.kafka.server.util.timer.SystemTimer import scala.jdk.CollectionConverters._ +import scala.jdk.OptionConverters._ object KafkaRaftManager { private def createLogDirectory(logDir: File, logDirName: String): File = { @@ -133,6 +132,8 @@ trait RaftManager[T] { def client: RaftClient[T] def replicatedLog: ReplicatedLog + + def voterNode(id: Int, listener: String): Option[Node] } class KafkaRaftManager[T]( @@ -313,4 +314,8 @@ class KafkaRaftManager[T]( override def leaderAndEpoch: LeaderAndEpoch = { client.leaderAndEpoch } + + override def voterNode(id: Int, listener: String): Option[Node] = { + client.voterNode(id, listener).toScala + } } diff --git a/core/src/main/scala/kafka/serializer/Decoder.scala b/core/src/main/scala/kafka/serializer/Decoder.scala index ce166689cfa6e..a0220a83913d5 100644 --- a/core/src/main/scala/kafka/serializer/Decoder.scala +++ b/core/src/main/scala/kafka/serializer/Decoder.scala @@ -27,13 +27,15 @@ import kafka.utils.VerifiableProperties * An implementation is required to provide a constructor that * takes a VerifiableProperties instance. */ -trait Decoder[T] { +@deprecated(since = "3.8.0") +trait Decoder[T] { def fromBytes(bytes: Array[Byte]): T } /** * The default implementation does nothing, just returns the same byte array it takes in. */ +@deprecated(since = "3.8.0") class DefaultDecoder(props: VerifiableProperties = null) extends Decoder[Array[Byte]] { def fromBytes(bytes: Array[Byte]): Array[Byte] = bytes } @@ -42,6 +44,7 @@ class DefaultDecoder(props: VerifiableProperties = null) extends Decoder[Array[B * The string decoder translates bytes into strings. It uses UTF8 by default but takes * an optional property serializer.encoding to control this. */ +@deprecated(since = "3.8.0") class StringDecoder(props: VerifiableProperties = null) extends Decoder[String] { val encoding: String = if (props == null) @@ -57,6 +60,7 @@ class StringDecoder(props: VerifiableProperties = null) extends Decoder[String] /** * The long decoder translates bytes into longs. */ +@deprecated(since = "3.8.0") class LongDecoder(props: VerifiableProperties = null) extends Decoder[Long] { def fromBytes(bytes: Array[Byte]): Long = { ByteBuffer.wrap(bytes).getLong @@ -66,6 +70,7 @@ class LongDecoder(props: VerifiableProperties = null) extends Decoder[Long] { /** * The integer decoder translates bytes into integers. */ +@deprecated(since = "3.8.0") class IntegerDecoder(props: VerifiableProperties = null) extends Decoder[Integer] { def fromBytes(bytes: Array[Byte]): Integer = { ByteBuffer.wrap(bytes).getInt() diff --git a/core/src/main/scala/kafka/server/ApiVersionManager.scala b/core/src/main/scala/kafka/server/ApiVersionManager.scala index 8b838a0c359b1..77ba7d69dee73 100644 --- a/core/src/main/scala/kafka/server/ApiVersionManager.scala +++ b/core/src/main/scala/kafka/server/ApiVersionManager.scala @@ -23,7 +23,7 @@ import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.ApiVersionsResponse import org.apache.kafka.server.ClientMetricsManager -import org.apache.kafka.server.common.Features +import org.apache.kafka.server.common.FinalizedFeatures import scala.collection.mutable import scala.jdk.CollectionConverters._ @@ -40,7 +40,7 @@ trait ApiVersionManager { } def newRequestMetrics: RequestChannel.Metrics = new network.RequestChannel.Metrics(enabledApis) - def features: Features + def features: FinalizedFeatures } object ApiVersionManager { @@ -73,6 +73,7 @@ object ApiVersionManager { * @param brokerFeatures the broker features * @param enableUnstableLastVersion whether to enable unstable last version, see [[KafkaConfig.unstableApiVersionsEnabled]] * @param zkMigrationEnabled whether to enable zk migration, see [[KafkaConfig.migrationEnabled]] + * @param featuresProvider a provider to the finalized features supported */ class SimpleApiVersionManager( val listenerType: ListenerType, @@ -80,14 +81,14 @@ class SimpleApiVersionManager( brokerFeatures: org.apache.kafka.common.feature.Features[SupportedVersionRange], val enableUnstableLastVersion: Boolean, val zkMigrationEnabled: Boolean, - val featuresProvider: () => Features + val featuresProvider: () => FinalizedFeatures ) extends ApiVersionManager { def this( listenerType: ListenerType, enableUnstableLastVersion: Boolean, zkMigrationEnabled: Boolean, - featuresProvider: () => Features + featuresProvider: () => FinalizedFeatures ) = { this( listenerType, @@ -113,7 +114,7 @@ class SimpleApiVersionManager( ) } - override def features: Features = featuresProvider.apply() + override def features: FinalizedFeatures = featuresProvider.apply() } /** @@ -164,5 +165,5 @@ class DefaultApiVersionManager( ) } - override def features: Features = metadataCache.features() + override def features: FinalizedFeatures = metadataCache.features() } diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index 41333c2bd3754..e685448de2134 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -19,6 +19,7 @@ package kafka.server import kafka.utils.Logging import org.apache.kafka.common.feature.{Features, SupportedVersionRange} +import org.apache.kafka.server.common.Features.PRODUCTION_FEATURES import org.apache.kafka.server.common.MetadataVersion import java.util @@ -70,21 +71,29 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte object BrokerFeatures extends Logging { - def createDefault(unstableMetadataVersionsEnabled: Boolean): BrokerFeatures = { - new BrokerFeatures(defaultSupportedFeatures(unstableMetadataVersionsEnabled)) + def createDefault(unstableFeatureVersionsEnabled: Boolean): BrokerFeatures = { + new BrokerFeatures(defaultSupportedFeatures(unstableFeatureVersionsEnabled)) } - def defaultSupportedFeatures(unstableMetadataVersionsEnabled: Boolean): Features[SupportedVersionRange] = { - Features.supportedFeatures( - java.util.Collections.singletonMap(MetadataVersion.FEATURE_NAME, + def defaultSupportedFeatures(unstableFeatureVersionsEnabled: Boolean): Features[SupportedVersionRange] = { + val features = new util.HashMap[String, SupportedVersionRange]() + features.put(MetadataVersion.FEATURE_NAME, new SupportedVersionRange( MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), - if (unstableMetadataVersionsEnabled) { + if (unstableFeatureVersionsEnabled) { MetadataVersion.latestTesting.featureLevel } else { MetadataVersion.latestProduction.featureLevel - } - ))) + })) + PRODUCTION_FEATURES.forEach { feature => features.put(feature.featureName, + new SupportedVersionRange(0, + if (unstableFeatureVersionsEnabled) { + feature.latestTesting + } else { + feature.latestProduction + })) + } + Features.supportedFeatures(features) } def createEmpty(): BrokerFeatures = { diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala index 1a10c6f659574..51bc16fb09d17 100644 --- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala +++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala @@ -33,6 +33,7 @@ import org.apache.kafka.queue.{EventQueue, KafkaEventQueue} import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} import java.util.{Comparator, OptionalLong} +import scala.collection.mutable import scala.jdk.CollectionConverters._ /** @@ -58,7 +59,8 @@ class BrokerLifecycleManager( val time: Time, val threadNamePrefix: String, val isZkBroker: Boolean, - val logDirs: Set[Uuid] + val logDirs: Set[Uuid], + val shutdownHook: () => Unit = () => {} ) extends Logging { private def logPrefix(): String = { @@ -149,10 +151,11 @@ class BrokerLifecycleManager( private var readyToUnfence = false /** - * List of accumulated offline directories. + * Map of accumulated offline directories. The value is true if the directory couldn't be communicated + * to the Controller. * This variable can only be read or written from the event queue thread. */ - private var offlineDirs = Set[Uuid]() + private var offlineDirs = mutable.Map[Uuid, Boolean]() /** * True if we sent a event queue to the active controller requesting controlled @@ -253,15 +256,19 @@ class BrokerLifecycleManager( * Propagate directory failures to the controller. * @param directory The ID for the directory that failed. */ - def propagateDirectoryFailure(directory: Uuid): Unit = { + def propagateDirectoryFailure(directory: Uuid, timeout: Long): Unit = { eventQueue.append(new OfflineDirEvent(directory)) + // If we can't communicate the offline directory to the controller, we should shut down. + eventQueue.scheduleDeferred("offlineDirFailure", + new DeadlineFunction(time.nanoseconds() + MILLISECONDS.toNanos(timeout)), + new OfflineDirBrokerFailureEvent(directory)) } - def handleKraftJBODMetadataVersionUpdate(): Unit = { - eventQueue.append(new KraftJBODMetadataVersionUpdateEvent()) + def resendBrokerRegistrationUnlessZkMode(): Unit = { + eventQueue.append(new ResendBrokerRegistrationUnlessZkModeEvent()) } - private class KraftJBODMetadataVersionUpdateEvent extends EventQueue.Event { + private class ResendBrokerRegistrationUnlessZkModeEvent extends EventQueue.Event { override def run(): Unit = { if (!isZkBroker) { registered = false @@ -327,9 +334,9 @@ class BrokerLifecycleManager( private class OfflineDirEvent(val dir: Uuid) extends EventQueue.Event { override def run(): Unit = { if (offlineDirs.isEmpty) { - offlineDirs = Set(dir) + offlineDirs = mutable.Map(dir -> false) } else { - offlineDirs = offlineDirs + dir + offlineDirs += (dir -> false) } if (registered) { scheduleNextCommunicationImmediately() @@ -337,6 +344,15 @@ class BrokerLifecycleManager( } } + private class OfflineDirBrokerFailureEvent(offlineDir: Uuid) extends EventQueue.Event { + override def run(): Unit = { + if (!offlineDirs.getOrElse(offlineDir, false)) { + error(s"Shutting down because couldn't communicate offline log dir $offlineDir with controllers") + shutdownHook() + } + } + } + private class StartupEvent(highestMetadataOffsetProvider: () => Long, channelManager: NodeToControllerChannelManager, clusterId: String, @@ -456,11 +472,11 @@ class BrokerLifecycleManager( setCurrentMetadataOffset(metadataOffset). setWantFence(!readyToUnfence). setWantShutDown(_state == BrokerState.PENDING_CONTROLLED_SHUTDOWN). - setOfflineLogDirs(offlineDirs.toSeq.asJava) + setOfflineLogDirs(offlineDirs.keys.toSeq.asJava) if (isTraceEnabled) { trace(s"Sending broker heartbeat $data") } - val handler = new BrokerHeartbeatResponseHandler() + val handler = new BrokerHeartbeatResponseHandler(offlineDirs.keys) _channelManager.sendRequest(new BrokerHeartbeatRequest.Builder(data), handler) communicationInFlight = true } @@ -468,18 +484,19 @@ class BrokerLifecycleManager( // the response handler is not invoked from the event handler thread, // so it is not safe to update state here, instead, schedule an event // to continue handling the response on the event handler thread - private class BrokerHeartbeatResponseHandler extends ControllerRequestCompletionHandler { + private class BrokerHeartbeatResponseHandler(currentOfflineDirs: Iterable[Uuid]) extends ControllerRequestCompletionHandler { override def onComplete(response: ClientResponse): Unit = { - eventQueue.prepend(new BrokerHeartbeatResponseEvent(response, false)) + eventQueue.prepend(new BrokerHeartbeatResponseEvent(response, false, currentOfflineDirs)) } override def onTimeout(): Unit = { info("Unable to send a heartbeat because the RPC got timed out before it could be sent.") - eventQueue.prepend(new BrokerHeartbeatResponseEvent(null, true)) + eventQueue.prepend(new BrokerHeartbeatResponseEvent(null, true, currentOfflineDirs)) } } - private class BrokerHeartbeatResponseEvent(response: ClientResponse, timedOut: Boolean) extends EventQueue.Event { + private class BrokerHeartbeatResponseEvent(response: ClientResponse, timedOut: Boolean, + currentOfflineDirs: Iterable[Uuid]) extends EventQueue.Event { override def run(): Unit = { communicationInFlight = false if (timedOut) { @@ -507,6 +524,7 @@ class BrokerLifecycleManager( if (errorCode == Errors.NONE) { val responseData = message.data() failedAttempts = 0 + currentOfflineDirs.foreach(cur => offlineDirs.put(cur, true)) _state match { case BrokerState.STARTING => if (responseData.isCaughtUp) { diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 94da0fe58375f..5e299fc0e02a8 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -34,12 +34,11 @@ import org.apache.kafka.common.network.ListenerName 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, KafkaException, TopicPartition, Uuid} +import org.apache.kafka.common.{ClusterResource, TopicPartition, Uuid} import org.apache.kafka.coordinator.group.metrics.{GroupCoordinatorMetrics, GroupCoordinatorRuntimeMetrics} import org.apache.kafka.coordinator.group.{CoordinatorRecord, GroupCoordinator, GroupCoordinatorConfig, GroupCoordinatorService, CoordinatorRecordSerde} -import org.apache.kafka.image.publisher.MetadataPublisher +import org.apache.kafka.image.publisher.{BrokerRegistrationTracker, MetadataPublisher} import org.apache.kafka.metadata.{BrokerState, ListenerInfo, VersionRange} -import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.{AssignmentsManager, ClientMetricsManager, NodeToControllerChannelManager} import org.apache.kafka.server.authorizer.Authorizer @@ -57,7 +56,7 @@ import java.util import java.util.Optional import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.{Condition, ReentrantLock} -import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit, TimeoutException} +import java.util.concurrent.{CompletableFuture, ExecutionException, TimeoutException, TimeUnit} import scala.collection.Map import scala.compat.java8.OptionConverters.RichOptionForJava8 import scala.jdk.CollectionConverters._ @@ -140,7 +139,9 @@ class BrokerServer( var brokerMetadataPublisher: BrokerMetadataPublisher = _ - val brokerFeatures: BrokerFeatures = BrokerFeatures.createDefault(config.unstableMetadataVersionsEnabled) + var brokerRegistrationTracker: BrokerRegistrationTracker = _ + + val brokerFeatures: BrokerFeatures = BrokerFeatures.createDefault(config.unstableFeatureVersionsEnabled) def kafkaYammerMetrics: KafkaYammerMetrics = KafkaYammerMetrics.INSTANCE @@ -185,7 +186,7 @@ class BrokerServer( kafkaScheduler.startup() /* register broker metrics */ - brokerTopicStats = new BrokerTopicStats(java.util.Optional.of(config)) + brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig.enableRemoteStorageSystem()) quotaManagers = QuotaFactory.instantiate(config, metrics, time, s"broker-${config.nodeId}-") @@ -210,19 +211,19 @@ class BrokerServer( time, s"broker-${config.nodeId}-", isZkBroker = false, - logDirs = logManager.directoryIdsSet) + logDirs = logManager.directoryIdsSet, + () => new Thread(() => shutdown(), "kafka-shutdown-thread").start()) // Enable delegation token cache for all SCRAM mechanisms to simplify dynamic update. // This keeps the cache up-to-date if new SCRAM mechanisms are enabled dynamically. tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames) credentialProvider = new CredentialProvider(ScramMechanism.mechanismNames, tokenCache) - val voterConnections = FutureUtils.waitWithLogging(logger.underlying, logIdent, + FutureUtils.waitWithLogging(logger.underlying, logIdent, "controller quorum voters future", sharedServer.controllerQuorumVotersFuture, startupDeadline, time) - val controllerNodes = QuorumConfig.voterConnectionsToNodes(voterConnections).asScala - val controllerNodeProvider = RaftControllerNodeProvider(raftManager, config, controllerNodes) + val controllerNodeProvider = RaftControllerNodeProvider(raftManager, config) clientToControllerChannelManager = new NodeToControllerChannelManagerImpl( controllerNodeProvider, @@ -304,7 +305,7 @@ class BrokerServer( assignmentsManager.onAssignment(partition, directoryId, reason, callback) override def handleFailure(directoryId: Uuid): Unit = - lifecycleManager.propagateDirectoryFailure(directoryId) + lifecycleManager.propagateDirectoryFailure(directoryId, config.logDirFailureTimeoutMs) } this._replicaManager = new ReplicaManager( @@ -429,6 +430,23 @@ class BrokerServer( config.numIoThreads, s"${DataPlaneAcceptor.MetricPrefix}RequestHandlerAvgIdlePercent", DataPlaneAcceptor.ThreadPrefix) + // Start RemoteLogManager before initializing broker metadata publishers. + remoteLogManagerOpt.foreach { rlm => + val listenerName = config.remoteLogManagerConfig.remoteLogMetadataManagerListenerName() + if (listenerName != null) { + val endpoint = listenerInfo.listeners().values().stream + .filter(e => + e.listenerName().isPresent && + ListenerName.normalised(e.listenerName().get()).equals(ListenerName.normalised(listenerName)) + ) + .findFirst() + .orElseThrow(() => new ConfigException(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_LISTENER_NAME_PROP, + listenerName, "Should be set as a listener name within valid broker listener name list: " + listenerInfo.listeners().values())) + rlm.onEndPointCreated(EndPoint.fromJava(endpoint)) + } + rlm.startup() + } + brokerMetadataPublisher = new BrokerMetadataPublisher(config, metadataCache, logManager, @@ -466,6 +484,10 @@ class BrokerServer( lifecycleManager ) metadataPublishers.add(brokerMetadataPublisher) + brokerRegistrationTracker = new BrokerRegistrationTracker(config.brokerId, + logManager.directoryIdsSet.toList.asJava, + () => lifecycleManager.resendBrokerRegistrationUnlessZkMode()) + metadataPublishers.add(brokerRegistrationTracker) // Register parts of the broker that can be reconfigured via dynamic configs. This needs to // be done before we publish the dynamic configs, so that we don't miss anything. @@ -498,23 +520,6 @@ class BrokerServer( // contain the original configuration values. new KafkaConfig(config.originals(), true) - // Start RemoteLogManager before broker start serving the requests. - remoteLogManagerOpt.foreach { rlm => - val listenerName = config.remoteLogManagerConfig.remoteLogMetadataManagerListenerName() - if (listenerName != null) { - val endpoint = listenerInfo.listeners().values().stream - .filter(e => - e.listenerName().isPresent && - ListenerName.normalised(e.listenerName().get()).equals(ListenerName.normalised(listenerName)) - ) - .findFirst() - .orElseThrow(() => new ConfigException(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_LISTENER_NAME_PROP, - listenerName, "Should be set as a listener name within valid broker listener name list: " + listenerInfo.listeners().values())) - rlm.onEndPointCreated(EndPoint.fromJava(endpoint)) - } - rlm.startup() - } - // We're now ready to unfence the broker. This also allows this broker to transition // from RECOVERY state to RUNNING state, once the controller unfences the broker. FutureUtils.waitWithLogging(logger.underlying, logIdent, @@ -615,10 +620,6 @@ class BrokerServer( protected def createRemoteLogManager(): Option[RemoteLogManager] = { if (config.remoteLogManagerConfig.enableRemoteStorageSystem()) { - if (config.logDirs.size > 1) { - throw new KafkaException("Tiered storage is not supported with multiple log dirs.") - } - Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time, (tp: TopicPartition) => logManager.getLog(tp).asJava, (tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => { @@ -626,7 +627,7 @@ class BrokerServer( log.updateLogStartOffsetFromRemoteTier(remoteLogStartOffset) } }, - brokerTopicStats)) + brokerTopicStats, metrics)) } else { None } diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index 1d5702e76e49d..ed9260b21947b 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -70,7 +70,7 @@ class TopicConfigHandler(private val replicaManager: ReplicaManager, val logs = logManager.logsByTopic(topic) val wasRemoteLogEnabledBeforeUpdate = logs.exists(_.remoteLogEnabled()) - logManager.updateTopicConfig(topic, props, kafkaConfig.isRemoteLogStorageSystemEnabled) + logManager.updateTopicConfig(topic, props, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem()) maybeBootstrapRemoteLogComponents(topic, logs, wasRemoteLogEnabledBeforeUpdate) } diff --git a/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala b/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala index 15eb1eff04aa3..f957b65ddd105 100644 --- a/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala +++ b/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala @@ -107,7 +107,8 @@ class ControllerConfigurationValidator(kafkaConfig: KafkaConfig) extends Configu throw new InvalidConfigurationException("Null value not supported for topic configs: " + nullTopicConfigs.mkString(",")) } - LogConfig.validate(properties, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled) + LogConfig.validate(properties, kafkaConfig.extractLogConfigMap, + kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem()) case BROKER => validateBrokerName(resource.name()) case CLIENT_METRICS => val properties = new Properties() diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index d0ea5387a26be..58a033ac63839 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -217,7 +217,7 @@ class ControllerServer( startupDeadline, time) val controllerNodes = QuorumConfig.voterConnectionsToNodes(voterConnections) val quorumFeatures = new QuorumFeatures(config.nodeId, - QuorumFeatures.defaultFeatureMap(config.unstableMetadataVersionsEnabled), + QuorumFeatures.defaultFeatureMap(config.unstableFeatureVersionsEnabled), controllerNodes.asScala.map(node => Integer.valueOf(node.id())).asJava) val delegationTokenKeyString = { @@ -349,7 +349,7 @@ class ControllerServer( clusterId, time, s"controller-${config.nodeId}-", - QuorumFeatures.defaultFeatureMap(config.unstableMetadataVersionsEnabled), + QuorumFeatures.defaultFeatureMap(config.unstableFeatureVersionsEnabled), config.migrationEnabled, incarnationId, listenerInfo) @@ -436,7 +436,7 @@ class ControllerServer( /** * Start the KIP-919 controller registration manager. */ - val controllerNodeProvider = RaftControllerNodeProvider(raftManager, config, controllerNodes.asScala) + val controllerNodeProvider = RaftControllerNodeProvider(raftManager, config) registrationChannelManager = new NodeToControllerChannelManagerImpl( controllerNodeProvider, time, diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index f8b60b6071d05..cc89631ab405f 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -91,19 +91,19 @@ class DelayedFetch( // Go directly to the check for Case G if the message offsets are the same. If the log segment // has just rolled, then the high watermark offset will remain the same but be on the old segment, // which would incorrectly be seen as an instance of Case F. - if (endOffset.messageOffset != fetchOffset.messageOffset) { - if (endOffset.onOlderSegment(fetchOffset)) { - // Case F, this can happen when the new fetch operation is on a truncated leader - debug(s"Satisfying fetch $this since it is fetching later segments of partition $topicIdPartition.") - return forceComplete() - } else if (fetchOffset.onOlderSegment(endOffset)) { + if (fetchOffset.messageOffset > endOffset.messageOffset) { + // Case F, this can happen when the new fetch operation is on a truncated leader + debug(s"Satisfying fetch $this since it is fetching later segments of partition $topicIdPartition.") + return forceComplete() + } else if (fetchOffset.messageOffset < endOffset.messageOffset) { + if (fetchOffset.onOlderSegment(endOffset)) { // Case F, this can happen when the fetch operation is falling behind the current segment // or the partition has just rolled a new segment debug(s"Satisfying fetch $this immediately since it is fetching older segments.") // We will not force complete the fetch request if a replica should be throttled. if (!params.isFromFollower || !replicaManager.shouldLeaderThrottle(quota, partition, params.replicaId)) return forceComplete() - } else if (fetchOffset.messageOffset < endOffset.messageOffset) { + } else if (fetchOffset.onSameSegment(endOffset)) { // we take the partition fetch size as upper bound when accumulating the bytes (skip if a throttled partition) val bytesAvailable = math.min(endOffset.positionDiff(fetchOffset), fetchStatus.fetchInfo.maxBytes) if (!params.isFromFollower || !replicaManager.shouldLeaderThrottle(quota, partition, params.replicaId)) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 4c320e7c6f5f8..c9bb2e3b4ffda 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -29,7 +29,8 @@ import kafka.utils.{CoreUtils, Logging} 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.config.internals.BrokerSecurityConfigs +import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, SaslConfigs, SslConfigs} import org.apache.kafka.common.metrics.{JmxReporter, Metrics, MetricsReporter} import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.network.{ListenerName, ListenerReconfigurable} @@ -39,7 +40,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.PasswordEncoder import org.apache.kafka.server.ProcessRole -import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZooKeeperInternals} +import org.apache.kafka.server.config.{ConfigType, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZooKeeperInternals} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.{ClientMetricsReceiverPlugin, MetricConfigs} import org.apache.kafka.server.telemetry.ClientTelemetry @@ -98,24 +99,26 @@ object DynamicBrokerConfig { DynamicProducerStateManagerConfig ++ DynamicRemoteLogConfig.ReconfigurableConfigs - private val ClusterLevelListenerConfigs = Set(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG, KafkaConfig.NumNetworkThreadsProp) + private val ClusterLevelListenerConfigs = Set(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG, ServerConfigs.NUM_NETWORK_THREADS_CONFIG) private val PerBrokerConfigs = (DynamicSecurityConfigs ++ DynamicListenerConfig.ReconfigurableConfigs).diff( ClusterLevelListenerConfigs) - private val ListenerMechanismConfigs = Set(KafkaSecurityConfigs.SASL_JAAS_CONFIG, - KafkaSecurityConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS_CONFIG, - KafkaSecurityConfigs.SASL_LOGIN_CLASS_CONFIG, - KafkaSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, - KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG) + private val ListenerMechanismConfigs = Set(SaslConfigs.SASL_JAAS_CONFIG, + SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS, + SaslConfigs.SASL_LOGIN_CLASS, + BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, + BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG) private val ReloadableFileConfigs = Set(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG) - val ListenerConfigRegex = """listener\.name\.[^.]*\.(.*)""".r + private val ListenerConfigRegex = """listener\.name\.[^.]*\.(.*)""".r private val DynamicPasswordConfigs = { val passwordConfigs = KafkaConfig.configKeys.filter(_._2.`type` == ConfigDef.Type.PASSWORD).keySet AllDynamicConfigs.intersect(passwordConfigs) } + private val nonDynamicProps: Set[String] = KafkaConfig.configNames.toSet -- DynamicConfig.Broker.names.asScala + def isPasswordConfig(name: String): Boolean = DynamicBrokerConfig.DynamicPasswordConfigs.exists(name.endsWith) def brokerConfigSynonyms(name: String, matchListenerOverride: Boolean): List[String] = { @@ -165,7 +168,7 @@ object DynamicBrokerConfig { } private def nonDynamicConfigs(props: Properties): Set[String] = { - props.asScala.keySet.intersect(DynamicConfig.Broker.nonDynamicProps) + props.asScala.keySet.intersect(nonDynamicProps) } private def securityConfigsWithoutListenerPrefix(props: Properties): Set[String] = { @@ -318,7 +321,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging } private def verifyReconfigurableConfigs(configNames: Set[String]): Unit = CoreUtils.inWriteLock(lock) { - val nonDynamic = configNames.filter(DynamicConfig.Broker.nonDynamicProps.contains) + val nonDynamic = configNames.intersect(nonDynamicProps) require(nonDynamic.isEmpty, s"Reconfigurable contains non-dynamic configs $nonDynamic") } @@ -673,11 +676,10 @@ trait BrokerReconfigurable { object DynamicLogConfig { // Exclude message.format.version for now since we need to check that the version // is supported on all brokers in the cluster. - @nowarn("cat=deprecation") - val ExcludedConfigs = Set(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG) - - val ReconfigurableConfigs = ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.values.asScala.toSet -- ExcludedConfigs - val KafkaConfigToLogConfigName = ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.asScala.map { case (k, v) => (v, k) } + val ReconfigurableConfigs: Set[String] = + ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.values.asScala.toSet - ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG + val KafkaConfigToLogConfigName: Map[String, String] = + ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.asScala.map { case (k, v) => (v, k) } } class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends BrokerReconfigurable with Logging { @@ -769,10 +771,10 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends Brok object DynamicThreadPool { val ReconfigurableConfigs = Set( - KafkaConfig.NumIoThreadsProp, + ServerConfigs.NUM_IO_THREADS_CONFIG, ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, - KafkaConfig.BackgroundThreadsProp) + ServerConfigs.BACKGROUND_THREADS_CONFIG) def validateReconfiguration(currentConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { newConfig.values.forEach { (k, v) => @@ -794,10 +796,10 @@ object DynamicThreadPool { def getValue(config: KafkaConfig, name: String): Int = { name match { - case KafkaConfig.NumIoThreadsProp => config.numIoThreads + case ServerConfigs.NUM_IO_THREADS_CONFIG => config.numIoThreads case ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG => config.numReplicaFetchers case ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG => config.numRecoveryThreadsPerDataDir - case KafkaConfig.BackgroundThreadsProp => config.backgroundThreads + case ServerConfigs.BACKGROUND_THREADS_CONFIG => config.backgroundThreads case n => throw new IllegalStateException(s"Unexpected config $n") } } @@ -806,7 +808,7 @@ object DynamicThreadPool { class ControllerDynamicThreadPool(controller: ControllerServer) extends BrokerReconfigurable { override def reconfigurableConfigs: Set[String] = { - Set(KafkaConfig.NumIoThreadsProp) + Set(ServerConfigs.NUM_IO_THREADS_CONFIG) } override def validateReconfiguration(newConfig: KafkaConfig): Unit = { @@ -899,7 +901,7 @@ class DynamicMetricsReporters(brokerId: Int, config: KafkaConfig, metrics: Metri 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 val propsOverride = Map[String, AnyRef](ServerConfigs.BROKER_ID_CONFIG -> brokerId.toString) private[server] val currentReporters = mutable.Map[String, MetricsReporter]() createReporters(config, clusterId, metricsReporterClasses(dynamicConfig.currentKafkaConfig.values()).asJava, Collections.emptyMap[String, Object]) @@ -970,46 +972,46 @@ object DynamicListenerConfig { SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, // SSL configs - KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, - KafkaSecurityConfigs.SSL_PROTOCOL_CONFIG, - KafkaSecurityConfigs.SSL_PROVIDER_CONFIG, - KafkaSecurityConfigs.SSL_CIPHER_SUITES_CONFIG, - KafkaSecurityConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, - KafkaSecurityConfigs.SSL_KEYSTORE_TYPE_CONFIG, - KafkaSecurityConfigs.SSL_KEYSTORE_LOCATION_CONFIG, - KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, - KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG, - KafkaSecurityConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, - KafkaSecurityConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, - KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, - KafkaSecurityConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, - KafkaSecurityConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, - KafkaSecurityConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, - KafkaSecurityConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG, - KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, - KafkaSecurityConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG, + BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, + SslConfigs.SSL_PROTOCOL_CONFIG, + SslConfigs.SSL_PROVIDER_CONFIG, + SslConfigs.SSL_CIPHER_SUITES_CONFIG, + SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, + SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, + SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, + SslConfigs.SSL_KEY_PASSWORD_CONFIG, + SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, + SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, + SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, + SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, + SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG, + BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, + SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG, // SASL configs - KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, - KafkaSecurityConfigs.SASL_JAAS_CONFIG, - KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, - KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG, - KafkaSecurityConfigs.SASL_KERBEROS_KINIT_CMD_CONFIG, - KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_CONFIG, - KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_CONFIG, - KafkaSecurityConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_CONFIG, - KafkaSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG, - KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_CONFIG, - KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_CONFIG, - KafkaSecurityConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_CONFIG, - KafkaSecurityConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_CONFIG, + BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, + SaslConfigs.SASL_JAAS_CONFIG, + BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, + SaslConfigs.SASL_KERBEROS_SERVICE_NAME, + SaslConfigs.SASL_KERBEROS_KINIT_CMD, + SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, + SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER, + SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN, + BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG, + SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR, + SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER, + SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS, + SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS, // Connection limit configs SocketServerConfigs.MAX_CONNECTIONS_CONFIG, SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG, // Network threads - KafkaConfig.NumNetworkThreadsProp + ServerConfigs.NUM_NETWORK_THREADS_CONFIG ) } diff --git a/core/src/main/scala/kafka/server/DynamicConfig.scala b/core/src/main/scala/kafka/server/DynamicConfig.scala index 5a5e38931790a..955103bea6f8e 100644 --- a/core/src/main/scala/kafka/server/DynamicConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicConfig.scala @@ -20,9 +20,6 @@ package kafka.server import java.net.{InetAddress, UnknownHostException} import java.util.Properties import org.apache.kafka.common.config.ConfigDef -import org.apache.kafka.common.config.ConfigDef.Importance._ -import org.apache.kafka.common.config.ConfigDef.Range._ -import org.apache.kafka.common.config.ConfigDef.Type._ import org.apache.kafka.server.config.{QuotaConfigs, ZooKeeperInternals} import java.util @@ -35,48 +32,44 @@ import scala.jdk.CollectionConverters._ object DynamicConfig { object Broker { - // Definitions - val brokerConfigDef = new ConfigDef() - // Round minimum value down, to make it easier for users. - .define(QuotaConfigs.LEADER_REPLICATION_THROTTLED_RATE_CONFIG, LONG, QuotaConfigs.QUOTA_BYTES_PER_SECOND_DEFAULT, atLeast(0), MEDIUM, QuotaConfigs.LEADER_REPLICATION_THROTTLED_RATE_DOC) - .define(QuotaConfigs.FOLLOWER_REPLICATION_THROTTLED_RATE_CONFIG, LONG, QuotaConfigs.QUOTA_BYTES_PER_SECOND_DEFAULT, atLeast(0), MEDIUM, QuotaConfigs.FOLLOWER_REPLICATION_THROTTLED_RATE_DOC) - .define(QuotaConfigs.REPLICA_ALTER_LOG_DIRS_IO_MAX_BYTES_PER_SECOND_CONFIG, LONG, QuotaConfigs.QUOTA_BYTES_PER_SECOND_DEFAULT, atLeast(0), MEDIUM, QuotaConfigs.REPLICA_ALTER_LOG_DIRS_IO_MAX_BYTES_PER_SECOND_DOC) - DynamicBrokerConfig.addDynamicConfigs(brokerConfigDef) - val nonDynamicProps = KafkaConfig.configNames.toSet -- brokerConfigDef.names.asScala - - def names = brokerConfigDef.names - - def validate(props: Properties) = DynamicConfig.validate(brokerConfigDef, props, customPropsAllowed = true) + private val brokerConfigs = QuotaConfigs.brokerQuotaConfigs() + DynamicBrokerConfig.addDynamicConfigs(brokerConfigs) + + def configKeys: util.Map[String, ConfigDef.ConfigKey] = brokerConfigs.configKeys + + def names: util.Set[String] = brokerConfigs.names + + def validate(props: Properties): util.Map[String, AnyRef] = DynamicConfig.validate(brokerConfigs, props, customPropsAllowed = true) } object Client { private val clientConfigs = QuotaConfigs.userAndClientQuotaConfigs() - def configKeys = clientConfigs.configKeys + def configKeys: util.Map[String, ConfigDef.ConfigKey] = clientConfigs.configKeys - def names = clientConfigs.names + def names: util.Set[String] = clientConfigs.names - def validate(props: Properties) = DynamicConfig.validate(clientConfigs, props, customPropsAllowed = false) + def validate(props: Properties): util.Map[String, AnyRef] = DynamicConfig.validate(clientConfigs, props, customPropsAllowed = false) } object User { private val userConfigs = QuotaConfigs.scramMechanismsPlusUserAndClientQuotaConfigs() - def configKeys = userConfigs.configKeys + def configKeys: util.Map[String, ConfigDef.ConfigKey] = userConfigs.configKeys - def names = userConfigs.names + def names: util.Set[String] = userConfigs.names - def validate(props: Properties) = DynamicConfig.validate(userConfigs, props, customPropsAllowed = false) + def validate(props: Properties): util.Map[String, AnyRef] = DynamicConfig.validate(userConfigs, props, customPropsAllowed = false) } object Ip { private val ipConfigs = QuotaConfigs.ipConfigs() - def configKeys = ipConfigs.configKeys + def configKeys: util.Map[String, ConfigDef.ConfigKey] = ipConfigs.configKeys - def names = ipConfigs.names + def names: util.Set[String] = ipConfigs.names - def validate(props: Properties) = DynamicConfig.validate(ipConfigs, props, customPropsAllowed = false) + def validate(props: Properties): util.Map[String, AnyRef] = DynamicConfig.validate(ipConfigs, props, customPropsAllowed = false) def isValidIpEntity(ip: String): Boolean = { if (ip != ZooKeeperInternals.DEFAULT_STRING) { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 70258e8dac522..b76ebff59cb78 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -67,10 +67,10 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.utils.{ProducerIdAndEpoch, Time} import org.apache.kafka.common.{Node, TopicIdPartition, TopicPartition, Uuid} -import org.apache.kafka.coordinator.group.GroupCoordinator +import org.apache.kafka.coordinator.group.{Group, GroupCoordinator} import org.apache.kafka.server.ClientMetricsManager import org.apache.kafka.server.authorizer._ -import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.common.{GroupVersion, MetadataVersion} import org.apache.kafka.server.common.MetadataVersion.{IBP_0_11_0_IV0, IBP_2_3_IV0} import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, FetchParams, FetchPartitionData} @@ -3797,10 +3797,15 @@ class KafkaApis(val requestChannel: RequestChannel, ) } + private def isConsumerGroupProtocolEnabled(): Boolean = { + val version = metadataCache.features().finalizedFeatures().getOrDefault(GroupVersion.FEATURE_NAME, 0.toShort) + config.groupCoordinatorRebalanceProtocols.contains(Group.GroupType.CONSUMER) && version >= GroupVersion.GV_1.featureLevel + } + def handleConsumerGroupHeartbeat(request: RequestChannel.Request): CompletableFuture[Unit] = { val consumerGroupHeartbeatRequest = request.body[ConsumerGroupHeartbeatRequest] - if (!config.isNewGroupCoordinatorEnabled) { + if (!isConsumerGroupProtocolEnabled()) { // The API is not supported by the "old" group coordinator (the default). If the // new one is not enabled, we fail directly here. requestHelper.sendMaybeThrottle(request, consumerGroupHeartbeatRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception)) @@ -3825,7 +3830,7 @@ class KafkaApis(val requestChannel: RequestChannel, def handleConsumerGroupDescribe(request: RequestChannel.Request): CompletableFuture[Unit] = { val consumerGroupDescribeRequest = request.body[ConsumerGroupDescribeRequest] - if (!config.isNewGroupCoordinatorEnabled) { + if (!isConsumerGroupProtocolEnabled()) { // The API is not supported by the "old" group coordinator (the default). If the // new one is not enabled, we fail directly here. requestHelper.sendMaybeThrottle(request, request.body[ConsumerGroupDescribeRequest].getErrorResponse(Errors.UNSUPPORTED_VERSION.exception)) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 9a228e014365a..94a7b349af927 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -17,18 +17,17 @@ package kafka.server -import java.{lang, util} +import java.util import java.util.concurrent.TimeUnit import java.util.{Collections, Properties} import kafka.cluster.EndPoint -import kafka.utils.CoreUtils.parseCsvList import kafka.utils.{CoreUtils, Logging} import kafka.utils.Implicits._ -import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.Reconfigurable import org.apache.kafka.common.compress.{GzipCompression, Lz4Compression, ZstdCompression} -import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, TopicConfig} -import org.apache.kafka.common.config.ConfigDef.{ConfigKey, ValidList} +import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, SecurityConfig, SslClientAuth, SslConfigs, TopicConfig} +import org.apache.kafka.common.config.ConfigDef.{CaseInsensitiveValidString, ConfigKey, ValidList, ValidString} +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.record.{CompressionType, LegacyRecord, Records, TimestampType} @@ -38,7 +37,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.ConsumerGroupMigrationPolicy import org.apache.kafka.coordinator.group.Group.GroupType import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.coordinator.group.assignor.PartitionAssignor +import org.apache.kafka.coordinator.group.assignor.ConsumerGroupPartitionAssignor import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.QuorumConfig @@ -48,7 +47,7 @@ import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.{MetadataVersion, MetadataVersionValidator} import org.apache.kafka.server.common.MetadataVersion._ -import org.apache.kafka.server.config.{Defaults, KRaftConfigs, KafkaSecurityConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs} +import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, ServerConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.MetricConfigs import org.apache.kafka.server.record.BrokerCompressionType @@ -93,120 +92,11 @@ object KafkaConfig { zooKeeperClientProperty(zkClientConfig, ZkConfigs.ZK_SSL_KEY_STORE_LOCATION_CONFIG).isDefined } - /** ********* General Configuration ***********/ - val BrokerIdGenerationEnableProp = "broker.id.generation.enable" - val MaxReservedBrokerIdProp = "reserved.broker.max.id" - val BrokerIdProp = "broker.id" - val MessageMaxBytesProp = "message.max.bytes" - val NumNetworkThreadsProp = "num.network.threads" - val NumIoThreadsProp = "num.io.threads" - val BackgroundThreadsProp = "background.threads" - val NumReplicaAlterLogDirsThreadsProp = "num.replica.alter.log.dirs.threads" - val QueuedMaxRequestsProp = "queued.max.requests" - val QueuedMaxBytesProp = "queued.max.request.bytes" - val RequestTimeoutMsProp = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG - val ConnectionSetupTimeoutMsProp = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG - val ConnectionSetupTimeoutMaxMsProp = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG - val DeleteTopicEnableProp = "delete.topic.enable" - val CompressionTypeProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_TYPE_CONFIG) - val CompressionGzipLevelProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG) - val CompressionLz4LevelProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG) - val CompressionZstdLevelProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG) - - /************* Authorizer Configuration ***********/ - val AuthorizerClassNameProp = "authorizer.class.name" - val EarlyStartListenersProp = "early.start.listeners" - - /***************** rack configuration *************/ - val RackProp = "broker.rack" - - /** ********* Controlled shutdown configuration ***********/ - val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries" - val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms" - val ControlledShutdownEnableProp = "controlled.shutdown.enable" - - /** ********* Fetch Configuration **************/ - val MaxIncrementalFetchSessionCacheSlots = "max.incremental.fetch.session.cache.slots" - val FetchMaxBytes = "fetch.max.bytes" - - /** ********* Request Limit Configuration **************/ - val MaxRequestPartitionSizeLimit = "max.request.partition.size.limit" - - /** ********* Delegation Token Configuration ****************/ - val DelegationTokenSecretKeyAliasProp = "delegation.token.master.key" - val DelegationTokenSecretKeyProp = "delegation.token.secret.key" - val DelegationTokenMaxLifeTimeProp = "delegation.token.max.lifetime.ms" - val DelegationTokenExpiryTimeMsProp = "delegation.token.expiry.time.ms" - val DelegationTokenExpiryCheckIntervalMsProp = "delegation.token.expiry.check.interval.ms" - - /** Internal Configurations **/ - val UnstableApiVersionsEnableProp = "unstable.api.versions.enable" - val UnstableMetadataVersionsEnableProp = "unstable.metadata.versions.enable" - - /* Documentation */ - /** ********* General Configuration ***********/ - val BrokerIdGenerationEnableDoc = s"Enable automatic broker id generation on the server. When enabled the value configured for $MaxReservedBrokerIdProp should be reviewed." - val MaxReservedBrokerIdDoc = "Max number that can be used for a broker.id" - val BrokerIdDoc = "The broker id for this server. If unset, a unique broker id will be generated." + - "To avoid conflicts between ZooKeeper generated broker id's and user configured broker id's, generated broker ids " + - "start from " + MaxReservedBrokerIdProp + " + 1." - val MessageMaxBytesDoc = TopicConfig.MAX_MESSAGE_BYTES_DOC + - s"This can be set per topic with the topic level ${TopicConfig.MAX_MESSAGE_BYTES_CONFIG} config." - val NumNetworkThreadsDoc = s"The number of threads that the server uses for receiving requests from the network and sending responses to the network. Noted: each listener (except for controller listener) creates its own thread pool." - val NumIoThreadsDoc = "The number of threads that the server uses for processing requests, which may include disk I/O" - val NumReplicaAlterLogDirsThreadsDoc = "The number of threads that can move replicas between log directories, which may include disk I/O" - val BackgroundThreadsDoc = "The number of threads to use for various background processing tasks" - val QueuedMaxRequestsDoc = "The number of queued requests allowed for data-plane, before blocking the network threads" - val QueuedMaxRequestBytesDoc = "The number of queued bytes allowed before no more requests are read" - val RequestTimeoutMsDoc = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC - val ConnectionSetupTimeoutMsDoc = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC - val ConnectionSetupTimeoutMaxMsDoc = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC - val DeleteTopicEnableDoc = "Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off" - val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the standard compression codecs " + - "('gzip', 'snappy', 'lz4', 'zstd'). It additionally accepts 'uncompressed' which is equivalent to no compression; and " + - "'producer' which means retain the original compression codec set by the producer." - val CompressionGzipLevelDoc = s"The compression level to use if $CompressionTypeProp is set to 'gzip'." - val CompressionLz4LevelDoc = s"The compression level to use if $CompressionTypeProp is set to 'lz4'." - val CompressionZstdLevelDoc = s"The compression level to use if $CompressionTypeProp is set to 'zstd'." - - /************* Authorizer Configuration ***********/ - val AuthorizerClassNameDoc = s"The fully qualified name of a class that implements ${classOf[Authorizer].getName}" + - " interface, which is used by the broker for authorization." - val EarlyStartListenersDoc = "A comma-separated list of listener names which may be started before the authorizer has finished " + - "initialization. This is useful when the authorizer is dependent on the cluster itself for bootstrapping, as is the case for " + - "the StandardAuthorizer (which stores ACLs in the metadata log.) By default, all listeners included in controller.listener.names " + - "will also be early start listeners. A listener should not appear in this list if it accepts external traffic." - - /************* Rack Configuration **************/ - val RackDoc = "Rack of the broker. This will be used in rack aware replication assignment for fault tolerance. Examples: RACK1, us-east-1d" - - /** ********* Controlled shutdown configuration ***********/ - val ControlledShutdownMaxRetriesDoc = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens" - val ControlledShutdownRetryBackoffMsDoc = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying." - val ControlledShutdownEnableDoc = "Enable controlled shutdown of the server." - - /** ********* Fetch Configuration **************/ - val MaxIncrementalFetchSessionCacheSlotsDoc = "The maximum number of total incremental fetch sessions that we will maintain. FetchSessionCache is sharded into 8 shards and the limit is equally divided among all shards. Sessions are allocated to each shard in round-robin. Only entries within a shard are considered eligible for eviction." - val FetchMaxBytesDoc = "The maximum number of bytes we will return for a fetch request. Must be at least 1024." - - /** ********* Request Limit Configuration **************/ - val MaxRequestPartitionSizeLimitDoc = "The maximum number of partitions can be served in one request." - - /** ********* Delegation Token Configuration ****************/ - val DelegationTokenSecretKeyAliasDoc = s"DEPRECATED: An alias for $DelegationTokenSecretKeyProp, which should be used instead of this config." - val DelegationTokenSecretKeyDoc = "Secret key to generate and verify delegation tokens. The same key must be configured across all the brokers. " + - " If using Kafka with KRaft, the key must also be set across all controllers. " + - " If the key is not set or set to empty string, brokers will disable the delegation token support." - val DelegationTokenMaxLifeTimeDoc = "The token has a maximum lifetime beyond which it cannot be renewed anymore. Default value 7 days." - val DelegationTokenExpiryTimeMsDoc = "The token validity time in milliseconds before the token needs to be renewed. Default value 1 day." - val DelegationTokenExpiryCheckIntervalDoc = "Scan interval to remove expired delegation tokens." - @nowarn("cat=deprecation") val configDef = { import ConfigDef.Importance._ import ConfigDef.Range._ import ConfigDef.Type._ - import ConfigDef.ValidString._ new ConfigDef() @@ -232,19 +122,19 @@ object KafkaConfig { .define(ZkConfigs.ZK_SSL_OCSP_ENABLE_CONFIG, BOOLEAN, ZkConfigs.ZK_SSL_OCSP_ENABLE, LOW, ZkConfigs.ZK_SSL_OCSP_ENABLE_DOC) /** ********* General Configuration ***********/ - .define(BrokerIdGenerationEnableProp, BOOLEAN, Defaults.BROKER_ID_GENERATION_ENABLE, MEDIUM, BrokerIdGenerationEnableDoc) - .define(MaxReservedBrokerIdProp, INT, Defaults.MAX_RESERVED_BROKER_ID, atLeast(0), MEDIUM, MaxReservedBrokerIdDoc) - .define(BrokerIdProp, INT, Defaults.BROKER_ID, HIGH, BrokerIdDoc) - .define(MessageMaxBytesProp, INT, LogConfig.DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), HIGH, MessageMaxBytesDoc) - .define(NumNetworkThreadsProp, INT, Defaults.NUM_NETWORK_THREADS, atLeast(1), HIGH, NumNetworkThreadsDoc) - .define(NumIoThreadsProp, INT, Defaults.NUM_IO_THREADS, atLeast(1), HIGH, NumIoThreadsDoc) - .define(NumReplicaAlterLogDirsThreadsProp, INT, null, HIGH, NumReplicaAlterLogDirsThreadsDoc) - .define(BackgroundThreadsProp, INT, Defaults.BACKGROUND_THREADS, atLeast(1), HIGH, BackgroundThreadsDoc) - .define(QueuedMaxRequestsProp, INT, Defaults.QUEUED_MAX_REQUESTS, atLeast(1), HIGH, QueuedMaxRequestsDoc) - .define(QueuedMaxBytesProp, LONG, Defaults.QUEUED_MAX_REQUEST_BYTES, MEDIUM, QueuedMaxRequestBytesDoc) - .define(RequestTimeoutMsProp, INT, Defaults.REQUEST_TIMEOUT_MS, HIGH, RequestTimeoutMsDoc) - .define(ConnectionSetupTimeoutMsProp, LONG, Defaults.CONNECTION_SETUP_TIMEOUT_MS, MEDIUM, ConnectionSetupTimeoutMsDoc) - .define(ConnectionSetupTimeoutMaxMsProp, LONG, Defaults.CONNECTION_SETUP_TIMEOUT_MAX_MS, MEDIUM, ConnectionSetupTimeoutMaxMsDoc) + .define(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, BOOLEAN, ServerConfigs.BROKER_ID_GENERATION_ENABLE_DEFAULT, MEDIUM, ServerConfigs.BROKER_ID_GENERATION_ENABLE_DOC) + .define(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG, INT, ServerConfigs.RESERVED_BROKER_MAX_ID_DEFAULT, atLeast(0), MEDIUM, ServerConfigs.RESERVED_BROKER_MAX_ID_DOC) + .define(ServerConfigs.BROKER_ID_CONFIG, INT, ServerConfigs.BROKER_ID_DEFAULT, HIGH, ServerConfigs.BROKER_ID_DOC) + .define(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, INT, LogConfig.DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), HIGH, ServerConfigs.MESSAGE_MAX_BYTES_DOC) + .define(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, INT, ServerConfigs.NUM_NETWORK_THREADS_DEFAULT, atLeast(1), HIGH, ServerConfigs.NUM_NETWORK_THREADS_DOC) + .define(ServerConfigs.NUM_IO_THREADS_CONFIG, INT, ServerConfigs.NUM_IO_THREADS_DEFAULT, atLeast(1), HIGH, ServerConfigs.NUM_IO_THREADS_DOC) + .define(ServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG, INT, null, HIGH, ServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_DOC) + .define(ServerConfigs.BACKGROUND_THREADS_CONFIG, INT, ServerConfigs.BACKGROUND_THREADS_DEFAULT, atLeast(1), HIGH, ServerConfigs.BACKGROUND_THREADS_DOC) + .define(ServerConfigs.QUEUED_MAX_REQUESTS_CONFIG, INT, ServerConfigs.QUEUED_MAX_REQUESTS_DEFAULT, atLeast(1), HIGH, ServerConfigs.QUEUED_MAX_REQUESTS_DOC) + .define(ServerConfigs.QUEUED_MAX_BYTES_CONFIG, LONG, ServerConfigs.QUEUED_MAX_REQUEST_BYTES_DEFAULT, MEDIUM, ServerConfigs.QUEUED_MAX_REQUEST_BYTES_DOC) + .define(ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG, INT, ServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT, HIGH, ServerConfigs.REQUEST_TIMEOUT_MS_DOC) + .define(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG, LONG, ServerConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS, MEDIUM, ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC) + .define(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG, LONG, ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS, MEDIUM, ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC) /* * KRaft mode configs. @@ -272,8 +162,8 @@ object KafkaConfig { MEDIUM, KRaftConfigs.MIGRATION_METADATA_MIN_BATCH_SIZE_DOC) /************* Authorizer Configuration ***********/ - .define(AuthorizerClassNameProp, STRING, Defaults.AUTHORIZER_CLASS_NAME, new ConfigDef.NonNullValidator(), LOW, AuthorizerClassNameDoc) - .define(EarlyStartListenersProp, STRING, null, HIGH, EarlyStartListenersDoc) + .define(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, STRING, ServerConfigs.AUTHORIZER_CLASS_NAME_DEFAULT, new ConfigDef.NonNullValidator(), LOW, ServerConfigs.AUTHORIZER_CLASS_NAME_DOC) + .define(ServerConfigs.EARLY_START_LISTENERS_CONFIG, STRING, null, HIGH, ServerConfigs.EARLY_START_LISTENERS_DOC) /** ********* Socket Server Configuration ***********/ .define(SocketServerConfigs.LISTENERS_CONFIG, STRING, SocketServerConfigs.LISTENERS_DEFAULT, HIGH, SocketServerConfigs.LISTENERS_DOC) @@ -292,7 +182,7 @@ object KafkaConfig { .define(SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG, INT, SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_DEFAULT, atLeast(0), LOW, SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_DOC) /************ Rack Configuration ******************/ - .define(RackProp, STRING, null, MEDIUM, RackDoc) + .define(ServerConfigs.BROKER_RACK_CONFIG, STRING, null, MEDIUM, ServerConfigs.BROKER_RACK_DOC) /** ********* Log Configuration ***********/ .define(ServerLogConfigs.NUM_PARTITIONS_CONFIG, INT, ServerLogConfigs.NUM_PARTITIONS_DEFAULT, atLeast(1), MEDIUM, ServerLogConfigs.NUM_PARTITIONS_DOC) @@ -337,7 +227,7 @@ object KafkaConfig { .define(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, BOOLEAN, ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_DEFAULT, HIGH, ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_DOC) .define(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, INT, ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DEFAULT, atLeast(1), HIGH, ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DOC) .define(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, STRING, ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DEFAULT, new MetadataVersionValidator(), MEDIUM, ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DOC) - .define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, STRING, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DEFAULT, ConfigDef.ValidString.in("CreateTime", "LogAppendTime"), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DOC) + .define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, STRING, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DEFAULT, ValidString.in("CreateTime", "LogAppendTime"), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DOC) .define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC) .define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DOC) .define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DOC) @@ -345,6 +235,7 @@ object KafkaConfig { .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, BOOLEAN, ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_DEFAULT, LOW, ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_DOC) .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) + .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) /** ********* Replication configuration ***********/ .define(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG, INT, ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_DEFAULT, MEDIUM, ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_DOC) @@ -366,16 +257,16 @@ object KafkaConfig { .define(ReplicationConfigs.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_CONFIG, INT, ReplicationConfigs.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_DEFAULT, HIGH, ReplicationConfigs.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_DOC) .define(ReplicationConfigs.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_CONFIG, LONG, ReplicationConfigs.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_DEFAULT, atLeast(1), HIGH, ReplicationConfigs.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_DOC) .define(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, BOOLEAN, LogConfig.DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE, HIGH, ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_DOC) - .define(ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG, STRING, ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_DEFAULT, in(Utils.enumOptions(classOf[SecurityProtocol]):_*), MEDIUM, ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_DOC) + .define(ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG, STRING, ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_DEFAULT, ValidString.in(Utils.enumOptions(classOf[SecurityProtocol]):_*), MEDIUM, ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_DOC) .define(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, STRING, ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_DEFAULT, new MetadataVersionValidator(), MEDIUM, ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_DOC) .define(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, STRING, null, MEDIUM, ReplicationConfigs.INTER_BROKER_LISTENER_NAME_DOC) .define(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG, STRING, null, MEDIUM, ReplicationConfigs.REPLICA_SELECTOR_CLASS_DOC) /** ********* Controlled shutdown configuration ***********/ - .define(ControlledShutdownMaxRetriesProp, INT, Defaults.CONTROLLED_SHUTDOWN_MAX_RETRIES, MEDIUM, ControlledShutdownMaxRetriesDoc) - .define(ControlledShutdownRetryBackoffMsProp, LONG, Defaults.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS, MEDIUM, ControlledShutdownRetryBackoffMsDoc) - .define(ControlledShutdownEnableProp, BOOLEAN, Defaults.CONTROLLED_SHUTDOWN_ENABLE, MEDIUM, ControlledShutdownEnableDoc) + .define(ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG, INT, ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_DEFAULT, MEDIUM, ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_DOC) + .define(ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG, LONG, ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_DEFAULT, MEDIUM, ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_DOC) + .define(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, BOOLEAN, ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_DEFAULT, MEDIUM, ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_DOC) /** ********* Group coordinator configuration ***********/ .define(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, INT, GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_DEFAULT, MEDIUM, GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_DOC) @@ -385,7 +276,7 @@ object KafkaConfig { /** New group coordinator configs */ .define(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, LIST, GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DEFAULT, - ConfigDef.ValidList.in(Utils.enumOptions(classOf[GroupType]):_*), MEDIUM, GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DOC) + ValidList.in(Utils.enumOptions(classOf[GroupType]):_*), MEDIUM, GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DOC) .define(GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG, INT, GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_DEFAULT, atLeast(1), MEDIUM, GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_DOC) // Internal configuration used by integration and system tests. .defineInternal(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG, BOOLEAN, GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_DEFAULT, null, MEDIUM, GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_DOC) @@ -399,7 +290,7 @@ object KafkaConfig { .define(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG, INT, GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_DEFAULT, atLeast(1), MEDIUM, GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_DOC) .define(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_CONFIG, INT, GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_DEFAULT, atLeast(1), MEDIUM, GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_DOC) .define(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, LIST, GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_DEFAULT, null, MEDIUM, GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_DOC) - .defineInternal(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, STRING, GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_DEFAULT, ConfigDef.CaseInsensitiveValidString.in(Utils.enumOptions(classOf[ConsumerGroupMigrationPolicy]): _*), MEDIUM, GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_DOC) + .define(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, STRING, GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_DEFAULT, CaseInsensitiveValidString.in(Utils.enumOptions(classOf[ConsumerGroupMigrationPolicy]): _*), MEDIUM, GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_DOC) /** ********* Offset management configuration ***********/ .define(GroupCoordinatorConfig.OFFSET_METADATA_MAX_SIZE_CONFIG, INT, GroupCoordinatorConfig.OFFSET_METADATA_MAX_SIZE_DEFAULT, HIGH, GroupCoordinatorConfig.OFFSET_METADATA_MAX_SIZE_DOC) @@ -412,11 +303,11 @@ object KafkaConfig { .define(GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_CONFIG, LONG, GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_DEFAULT, atLeast(1), HIGH, GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_DOC) .define(GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG, INT, GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT, atLeast(1), HIGH, GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_DOC) .define(GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_CONFIG, SHORT, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DEFAULT, HIGH, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DOC) - .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DELETE_TOPIC_ENABLE, HIGH, DeleteTopicEnableDoc) - .define(CompressionTypeProp, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, CompressionTypeDoc) - .define(CompressionGzipLevelProp, INT, GzipCompression.DEFAULT_LEVEL, new GzipCompression.LevelValidator(), MEDIUM, CompressionGzipLevelDoc) - .define(CompressionLz4LevelProp, INT, Lz4Compression.DEFAULT_LEVEL, between(Lz4Compression.MIN_LEVEL, Lz4Compression.MAX_LEVEL), MEDIUM, CompressionLz4LevelDoc) - .define(CompressionZstdLevelProp, INT, ZstdCompression.DEFAULT_LEVEL, between(ZstdCompression.MIN_LEVEL, ZstdCompression.MAX_LEVEL), MEDIUM, CompressionZstdLevelDoc) + .define(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, BOOLEAN, ServerConfigs.DELETE_TOPIC_ENABLE_DEFAULT, HIGH, ServerConfigs.DELETE_TOPIC_ENABLE_DOC) + .define(ServerConfigs.COMPRESSION_TYPE_CONFIG, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, ValidString.in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, ServerConfigs.COMPRESSION_TYPE_DOC) + .define(ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG, INT, GzipCompression.DEFAULT_LEVEL, new GzipCompression.LevelValidator(), MEDIUM, ServerConfigs.COMPRESSION_GZIP_LEVEL_DOC) + .define(ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG, INT, Lz4Compression.DEFAULT_LEVEL, between(Lz4Compression.MIN_LEVEL, Lz4Compression.MAX_LEVEL), MEDIUM, ServerConfigs.COMPRESSION_LZ4_LEVEL_DOC) + .define(ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG, INT, ZstdCompression.DEFAULT_LEVEL, between(ZstdCompression.MIN_LEVEL, ZstdCompression.MAX_LEVEL), MEDIUM, ServerConfigs.COMPRESSION_ZSTD_LEVEL_DOC) /** ********* Transaction management configuration ***********/ .define(TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_CONFIG, INT, TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_DEFAULT, atLeast(1), HIGH, TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_DOC) @@ -436,11 +327,11 @@ object KafkaConfig { .defineInternal(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_CONFIG, INT, TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT, atLeast(1), LOW, TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DOC) /** ********* Fetch Configuration **************/ - .define(MaxIncrementalFetchSessionCacheSlots, INT, Defaults.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS, atLeast(0), MEDIUM, MaxIncrementalFetchSessionCacheSlotsDoc) - .define(FetchMaxBytes, INT, Defaults.FETCH_MAX_BYTES, atLeast(1024), MEDIUM, FetchMaxBytesDoc) + .define(ServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_CONFIG, INT, ServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_DEFAULT, atLeast(0), MEDIUM, ServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_DOC) + .define(ServerConfigs.FETCH_MAX_BYTES_CONFIG, INT, ServerConfigs.FETCH_MAX_BYTES_DEFAULT, atLeast(1024), MEDIUM, ServerConfigs.FETCH_MAX_BYTES_DOC) /** ********* Request Limit Configuration ***********/ - .define(MaxRequestPartitionSizeLimit, INT, Defaults.MAX_REQUEST_PARTITION_SIZE_LIMIT, atLeast(1), MEDIUM, MaxRequestPartitionSizeLimitDoc) + .define(ServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_CONFIG, INT, ServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_DEFAULT, atLeast(1), MEDIUM, ServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_DOC) /** ********* Kafka Metrics Configuration ***********/ .define(MetricConfigs.METRIC_NUM_SAMPLES_CONFIG, INT, MetricConfigs.METRIC_NUM_SAMPLES_DEFAULT, atLeast(1), LOW, MetricConfigs.METRIC_NUM_SAMPLES_DOC) @@ -468,75 +359,75 @@ object KafkaConfig { .define(QuotaConfigs.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, CLASS, null, LOW, QuotaConfigs.CLIENT_QUOTA_CALLBACK_CLASS_DOC) /** ********* General Security Configuration ****************/ - .define(KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, LONG, KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_DEFAULT, MEDIUM, KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_DOC) - .define(KafkaSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG, INT, KafkaSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_DOC) - .define(KafkaSecurityConfigs.SECURITY_PROVIDER_CLASS_CONFIG, STRING, null, LOW, KafkaSecurityConfigs.SECURITY_PROVIDERS_DOC) + .define(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, LONG, BrokerSecurityConfigs.DEFAULT_CONNECTIONS_MAX_REAUTH_MS, MEDIUM, BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_DOC) + .define(BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG, INT, BrokerSecurityConfigs.DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE, MEDIUM, BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_DOC) + .define(SecurityConfig.SECURITY_PROVIDERS_CONFIG, STRING, null, LOW, SecurityConfig.SECURITY_PROVIDERS_DOC) /** ********* SSL Configuration ****************/ - .define(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, CLASS, KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DEFAULT, MEDIUM, KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DOC) - .define(KafkaSecurityConfigs.SSL_PROTOCOL_CONFIG, STRING, KafkaSecurityConfigs.SSL_PROTOCOL_DEFAULT, MEDIUM, KafkaSecurityConfigs.SSL_PROTOCOL_DOC) - .define(KafkaSecurityConfigs.SSL_PROVIDER_CONFIG, STRING, null, MEDIUM, KafkaSecurityConfigs.SSL_PROVIDER_DOC) - .define(KafkaSecurityConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, LIST, KafkaSecurityConfigs.SSL_ENABLED_PROTOCOLS_DEFAULTS, MEDIUM, KafkaSecurityConfigs.SSL_ENABLED_PROTOCOLS_DOC) - .define(KafkaSecurityConfigs.SSL_KEYSTORE_TYPE_CONFIG, STRING, KafkaSecurityConfigs.SSL_KEYSTORE_TYPE_DEFAULT, MEDIUM, KafkaSecurityConfigs.SSL_KEYSTORE_TYPE_DOC) - .define(KafkaSecurityConfigs.SSL_KEYSTORE_LOCATION_CONFIG, STRING, null, MEDIUM, KafkaSecurityConfigs.SSL_KEYSTORE_LOCATION_DOC) - .define(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_DOC) - .define(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SSL_KEY_PASSWORD_DOC) - .define(KafkaSecurityConfigs.SSL_KEYSTORE_KEY_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SSL_KEYSTORE_KEY_DOC) - .define(KafkaSecurityConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_DOC) - .define(KafkaSecurityConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, STRING, KafkaSecurityConfigs.SSL_TRUSTSTORE_TYPE_DEFAULT, MEDIUM, KafkaSecurityConfigs.SSL_TRUSTSTORE_TYPE_DOC) - .define(KafkaSecurityConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, STRING, null, MEDIUM, KafkaSecurityConfigs.SSL_TRUSTSTORE_LOCATION_DOC) - .define(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_DOC) - .define(KafkaSecurityConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SSL_TRUSTSTORE_CERTIFICATES_DOC) - .define(KafkaSecurityConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, STRING, KafkaSecurityConfigs.SSL_KEYMANAGER_ALGORITHM_DEFAULT, MEDIUM, KafkaSecurityConfigs.SSL_KEYMANAGER_ALGORITHM_DOC) - .define(KafkaSecurityConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, STRING, KafkaSecurityConfigs.SSL_TRUSTMANAGER_ALGORITHM_DEFAULT, MEDIUM, KafkaSecurityConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC) - .define(KafkaSecurityConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, STRING, KafkaSecurityConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DEFAULT, LOW, KafkaSecurityConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC) - .define(KafkaSecurityConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG, STRING, null, LOW, KafkaSecurityConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_DOC) - .define(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, STRING, KafkaSecurityConfigs.SSL_CLIENT_AUTH_DEFAULT, in(KafkaSecurityConfigs.SSL_CLIENT_AUTHENTICATION_VALID_VALUES:_*), MEDIUM, KafkaSecurityConfigs.SSL_CLIENT_AUTH_DOC) - .define(KafkaSecurityConfigs.SSL_CIPHER_SUITES_CONFIG, LIST, Collections.emptyList(), MEDIUM, KafkaSecurityConfigs.SSL_CIPHER_SUITES_DOC) - .define(KafkaSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_CONFIG, STRING, KafkaSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_DEFAULT, LOW, KafkaSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_DOC) - .define(KafkaSecurityConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG, CLASS, null, LOW, KafkaSecurityConfigs.SSL_ENGINE_FACTORY_CLASS_DOC) - .define(KafkaSecurityConfigs.SSL_ALLOW_DN_CHANGES_CONFIG, BOOLEAN, KafkaSecurityConfigs.SSL_ALLOW_DN_CHANGES_DEFAULT, LOW, KafkaSecurityConfigs.SSL_ALLOW_DN_CHANGES_DOC) - .define(KafkaSecurityConfigs.SSL_ALLOW_SAN_CHANGES_CONFIG, BOOLEAN, KafkaSecurityConfigs.SSL_ALLOW_SAN_CHANGES_DEFAULT, LOW, KafkaSecurityConfigs.SSL_ALLOW_SAN_CHANGES_DOC) + .define(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, CLASS, BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DEFAULT, MEDIUM, BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DOC) + .define(SslConfigs.SSL_PROTOCOL_CONFIG, STRING, SslConfigs.DEFAULT_SSL_PROTOCOL, MEDIUM, SslConfigs.SSL_PROTOCOL_DOC) + .define(SslConfigs.SSL_PROVIDER_CONFIG, STRING, null, MEDIUM, SslConfigs.SSL_PROVIDER_DOC) + .define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC) + .define(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, STRING, SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, MEDIUM, SslConfigs.SSL_KEYSTORE_TYPE_DOC) + .define(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, STRING, null, MEDIUM, SslConfigs.SSL_KEYSTORE_LOCATION_DOC) + .define(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_KEYSTORE_PASSWORD_DOC) + .define(SslConfigs.SSL_KEY_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_KEY_PASSWORD_DOC) + .define(SslConfigs.SSL_KEYSTORE_KEY_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_KEYSTORE_KEY_DOC) + .define(SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_DOC) + .define(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, STRING, SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, MEDIUM, SslConfigs.SSL_TRUSTSTORE_TYPE_DOC) + .define(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, STRING, null, MEDIUM, SslConfigs.SSL_TRUSTSTORE_LOCATION_DOC) + .define(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_TRUSTSTORE_PASSWORD_DOC) + .define(SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_DOC) + .define(SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, STRING, SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, MEDIUM, SslConfigs.SSL_KEYMANAGER_ALGORITHM_DOC) + .define(SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, STRING, SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, MEDIUM, SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC) + .define(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, STRING, SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM, LOW, SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC) + .define(SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG, STRING, null, LOW, SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_DOC) + .define(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, STRING, BrokerSecurityConfigs.SSL_CLIENT_AUTH_DEFAULT, ValidString.in(Utils.enumOptions(classOf[SslClientAuth]):_*), MEDIUM, BrokerSecurityConfigs.SSL_CLIENT_AUTH_DOC) + .define(SslConfigs.SSL_CIPHER_SUITES_CONFIG, LIST, Collections.emptyList(), MEDIUM, SslConfigs.SSL_CIPHER_SUITES_DOC) + .define(BrokerSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_CONFIG, STRING, BrokerSecurityConfigs.DEFAULT_SSL_PRINCIPAL_MAPPING_RULES, LOW, BrokerSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_DOC) + .define(SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG, CLASS, null, LOW, SslConfigs.SSL_ENGINE_FACTORY_CLASS_DOC) + .define(BrokerSecurityConfigs.SSL_ALLOW_DN_CHANGES_CONFIG, BOOLEAN, BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_DN_CHANGES_VALUE, LOW, BrokerSecurityConfigs.SSL_ALLOW_DN_CHANGES_DOC) + .define(BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_CONFIG, BOOLEAN, BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE, LOW, BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_DOC) /** ********* Sasl Configuration ****************/ - .define(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, STRING, KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_DOC) - .define(KafkaSecurityConfigs.SASL_JAAS_CONFIG, PASSWORD, null, MEDIUM, KafkaSecurityConfigs.SASL_JAAS_CONFIG_DOC) - .define(KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, LIST, KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_DOC) - .define(KafkaSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, CLASS, null, MEDIUM, KafkaSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_DOC) - .define(KafkaSecurityConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_CONFIG, CLASS, null, MEDIUM, KafkaSecurityConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_DOC) - .define(KafkaSecurityConfigs.SASL_LOGIN_CLASS_CONFIG, CLASS, null, MEDIUM, KafkaSecurityConfigs.SASL_LOGIN_CLASS_DOC) - .define(KafkaSecurityConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS_CONFIG, CLASS, null, MEDIUM, KafkaSecurityConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS_DOC) - .define(KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG, STRING, null, MEDIUM, KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_DOC) - .define(KafkaSecurityConfigs.SASL_KERBEROS_KINIT_CMD_CONFIG, STRING, KafkaSecurityConfigs.SASL_KERBEROS_KINIT_CMD_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_KERBEROS_KINIT_CMD_DOC) - .define(KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_CONFIG, DOUBLE, KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC) - .define(KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_CONFIG, DOUBLE, KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DOC) - .define(KafkaSecurityConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_CONFIG, LONG, KafkaSecurityConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC) - .define(KafkaSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG, LIST, KafkaSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DOC) - .define(KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_CONFIG, DOUBLE, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC) - .define(KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_CONFIG, DOUBLE, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC) - .define(KafkaSecurityConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_CONFIG, SHORT, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC) - .define(KafkaSecurityConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_CONFIG, SHORT, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_DEFAULT, MEDIUM, KafkaSecurityConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC) - .define(KafkaSecurityConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_CONFIG, INT, null, LOW, KafkaSecurityConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC) - .define(KafkaSecurityConfigs.SASL_LOGIN_READ_TIMEOUT_MS_CONFIG, INT, null, LOW, KafkaSecurityConfigs.SASL_LOGIN_READ_TIMEOUT_MS_DOC) - .define(KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_CONFIG, LONG, KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DEFAULT, LOW, KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC) - .define(KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_CONFIG, LONG, KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_DEFAULT, LOW, KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_DOC) - .define(KafkaSecurityConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_CONFIG, STRING, KafkaSecurityConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DEFAULT, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC) - .define(KafkaSecurityConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_CONFIG, STRING, KafkaSecurityConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_DEFAULT, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC) - .define(KafkaSecurityConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_CONFIG, STRING, null, MEDIUM, KafkaSecurityConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC) - .define(KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_CONFIG, STRING, null, MEDIUM, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC) - .define(KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_CONFIG, LONG, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DEFAULT, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC) - .define(KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_CONFIG, LONG, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DEFAULT, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC) - .define(KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_CONFIG, LONG, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DEFAULT, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC) - .define(KafkaSecurityConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_CONFIG, INT, KafkaSecurityConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DEFAULT, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC) - .define(KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_CONFIG, LIST, null, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC) - .define(KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_CONFIG, STRING, null, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC) + .define(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, MEDIUM, BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_DOC) + .define(SaslConfigs.SASL_JAAS_CONFIG, PASSWORD, null, MEDIUM, SaslConfigs.SASL_JAAS_CONFIG_DOC) + .define(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, LIST, BrokerSecurityConfigs.DEFAULT_SASL_ENABLED_MECHANISMS, MEDIUM, BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_DOC) + .define(BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG, CLASS, null, MEDIUM, BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_DOC) + .define(SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS, CLASS, null, MEDIUM, SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_DOC) + .define(SaslConfigs.SASL_LOGIN_CLASS, CLASS, null, MEDIUM, SaslConfigs.SASL_LOGIN_CLASS_DOC) + .define(SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS, CLASS, null, MEDIUM, SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS_DOC) + .define(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, STRING, null, MEDIUM, SaslConfigs.SASL_KERBEROS_SERVICE_NAME_DOC) + .define(SaslConfigs.SASL_KERBEROS_KINIT_CMD, STRING, SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD, MEDIUM, SaslConfigs.SASL_KERBEROS_KINIT_CMD_DOC) + .define(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, DOUBLE, SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, MEDIUM, SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC) + .define(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER, DOUBLE, SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER, MEDIUM, SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DOC) + .define(SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN, LONG, SaslConfigs.DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN, MEDIUM, SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC) + .define(BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG, LIST, BrokerSecurityConfigs.DEFAULT_SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES, MEDIUM, BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DOC) + .define(SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR, DOUBLE, SaslConfigs.DEFAULT_LOGIN_REFRESH_WINDOW_FACTOR, MEDIUM, SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC) + .define(SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER, DOUBLE, SaslConfigs.DEFAULT_LOGIN_REFRESH_WINDOW_JITTER, MEDIUM, SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC) + .define(SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS, SHORT, SaslConfigs.DEFAULT_LOGIN_REFRESH_MIN_PERIOD_SECONDS, MEDIUM, SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC) + .define(SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS, SHORT, SaslConfigs.DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS, MEDIUM, SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC) + .define(SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS, INT, null, LOW, SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC) + .define(SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS, INT, null, LOW, SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS_DOC) + .define(SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS, LONG, SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MAX_MS, LOW, SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC) + .define(SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS, LONG, SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MS, LOW, SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, STRING, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME, LOW, SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME, STRING, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME, LOW, SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, STRING, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL, STRING, null, MEDIUM, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS, LONG, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS, LOW, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, LONG, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, LOW, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, LONG, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, LOW, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, INT, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, LOW, SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, LIST, null, LOW, SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER, STRING, null, LOW, SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC) /** ********* Delegation Token Configuration ****************/ - .define(DelegationTokenSecretKeyAliasProp, PASSWORD, null, MEDIUM, DelegationTokenSecretKeyAliasDoc) - .define(DelegationTokenSecretKeyProp, PASSWORD, null, MEDIUM, DelegationTokenSecretKeyDoc) - .define(DelegationTokenMaxLifeTimeProp, LONG, Defaults.DELEGATION_TOKEN_MAX_LIFE_TIME_MS, atLeast(1), MEDIUM, DelegationTokenMaxLifeTimeDoc) - .define(DelegationTokenExpiryTimeMsProp, LONG, Defaults.DELEGATION_TOKEN_EXPIRY_TIME_MS, atLeast(1), MEDIUM, DelegationTokenExpiryTimeMsDoc) - .define(DelegationTokenExpiryCheckIntervalMsProp, LONG, Defaults.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS, atLeast(1), LOW, DelegationTokenExpiryCheckIntervalDoc) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG, PASSWORD, null, MEDIUM, DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_DOC) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, PASSWORD, null, MEDIUM, DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_DOC) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFETIME_CONFIG, LONG, DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFE_TIME_MS_DEFAULT, atLeast(1), MEDIUM, DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFE_TIME_DOC) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG, LONG, DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_DEFAULT, atLeast(1), MEDIUM, DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_DOC) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG, LONG, DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_DEFAULT, atLeast(1), LOW, DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_DOC) /** ********* Password encryption configuration for dynamic configs *********/ .define(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, PASSWORD, null, MEDIUM, PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_DOC) @@ -557,9 +448,9 @@ object KafkaConfig { /** Internal Configurations **/ // This indicates whether unreleased APIs should be advertised by this node. - .defineInternal(UnstableApiVersionsEnableProp, BOOLEAN, false, HIGH) - // This indicates whether unreleased MetadataVersions should be enabled on this node. - .defineInternal(UnstableMetadataVersionsEnableProp, BOOLEAN, false, HIGH) + .defineInternal(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, BOOLEAN, false, HIGH) + // This indicates whether unreleased MetadataVersions or other feature versions should be enabled on this node. + .defineInternal(ServerConfigs.UNSTABLE_FEATURE_VERSIONS_ENABLE_CONFIG, BOOLEAN, false, HIGH) } /** ********* Remote Log Management Configuration *********/ @@ -606,7 +497,7 @@ object KafkaConfig { if (configType != null) { Some(configType) } else { - val configKey = DynamicConfig.Broker.brokerConfigDef.configKeys().get(exactName) + val configKey = DynamicConfig.Broker.configKeys.get(exactName) if (configKey != null) { Some(configKey.`type`) } else { @@ -636,10 +527,10 @@ object KafkaConfig { */ def populateSynonyms(input: util.Map[_, _]): util.Map[Any, Any] = { val output = new util.HashMap[Any, Any](input) - val brokerId = output.get(KafkaConfig.BrokerIdProp) + val brokerId = output.get(ServerConfigs.BROKER_ID_CONFIG) val nodeId = output.get(KRaftConfigs.NODE_ID_CONFIG) if (brokerId == null && nodeId != null) { - output.put(KafkaConfig.BrokerIdProp, nodeId) + output.put(ServerConfigs.BROKER_ID_CONFIG, nodeId) } else if (brokerId != null && nodeId == null) { output.put(KRaftConfigs.NODE_ID_CONFIG, brokerId) } @@ -773,9 +664,9 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val ZkSslCrlEnable = zkBooleanConfigOrSystemPropertyWithDefaultValue(ZkConfigs.ZK_SSL_CRL_ENABLE_CONFIG) val ZkSslOcspEnable = zkBooleanConfigOrSystemPropertyWithDefaultValue(ZkConfigs.ZK_SSL_OCSP_ENABLE_CONFIG) /** ********* General Configuration ***********/ - val brokerIdGenerationEnable: Boolean = getBoolean(KafkaConfig.BrokerIdGenerationEnableProp) - val maxReservedBrokerId: Int = getInt(KafkaConfig.MaxReservedBrokerIdProp) - var brokerId: Int = getInt(KafkaConfig.BrokerIdProp) + val brokerIdGenerationEnable: Boolean = getBoolean(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG) + val maxReservedBrokerId: Int = getInt(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG) + var brokerId: Int = getInt(ServerConfigs.BROKER_ID_CONFIG) val nodeId: Int = getInt(KRaftConfigs.NODE_ID_CONFIG) val initialRegistrationTimeoutMs: Int = getInt(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG) val brokerHeartbeatIntervalMs: Int = getInt(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG) @@ -825,18 +716,18 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami def metadataLogSegmentMinBytes = getInt(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG) val serverMaxStartupTimeMs = getLong(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG) - def numNetworkThreads = getInt(KafkaConfig.NumNetworkThreadsProp) - def backgroundThreads = getInt(KafkaConfig.BackgroundThreadsProp) - val queuedMaxRequests = getInt(KafkaConfig.QueuedMaxRequestsProp) - val queuedMaxBytes = getLong(KafkaConfig.QueuedMaxBytesProp) - def numIoThreads = getInt(KafkaConfig.NumIoThreadsProp) - def messageMaxBytes = getInt(KafkaConfig.MessageMaxBytesProp) - val requestTimeoutMs = getInt(KafkaConfig.RequestTimeoutMsProp) - val connectionSetupTimeoutMs = getLong(KafkaConfig.ConnectionSetupTimeoutMsProp) - val connectionSetupTimeoutMaxMs = getLong(KafkaConfig.ConnectionSetupTimeoutMaxMsProp) + def numNetworkThreads = getInt(ServerConfigs.NUM_NETWORK_THREADS_CONFIG) + def backgroundThreads = getInt(ServerConfigs.BACKGROUND_THREADS_CONFIG) + val queuedMaxRequests = getInt(ServerConfigs.QUEUED_MAX_REQUESTS_CONFIG) + val queuedMaxBytes = getLong(ServerConfigs.QUEUED_MAX_BYTES_CONFIG) + def numIoThreads = getInt(ServerConfigs.NUM_IO_THREADS_CONFIG) + def messageMaxBytes = getInt(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG) + val requestTimeoutMs = getInt(ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG) + val connectionSetupTimeoutMs = getLong(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG) + val connectionSetupTimeoutMaxMs = getLong(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG) def getNumReplicaAlterLogDirsThreads: Int = { - val numThreads: Integer = Option(getInt(KafkaConfig.NumReplicaAlterLogDirsThreadsProp)).getOrElse(logDirs.size) + val numThreads: Integer = Option(getInt(ServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG)).getOrElse(logDirs.size) numThreads } @@ -850,7 +741,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami /************* Authorizer Configuration ***********/ def createNewAuthorizer(): Option[Authorizer] = { - val className = getString(KafkaConfig.AuthorizerClassNameProp) + val className = getString(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG) if (className == null || className.isEmpty) None else { @@ -861,13 +752,13 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val earlyStartListeners: Set[ListenerName] = { val listenersSet = listeners.map(_.listenerName).toSet val controllerListenersSet = controllerListeners.map(_.listenerName).toSet - Option(getString(KafkaConfig.EarlyStartListenersProp)) match { + Option(getString(ServerConfigs.EARLY_START_LISTENERS_CONFIG)) match { case None => controllerListenersSet case Some(str) => str.split(",").map(_.trim()).filterNot(_.isEmpty).map { str => val listenerName = new ListenerName(str) if (!listenersSet.contains(listenerName) && !controllerListenersSet.contains(listenerName)) - throw new ConfigException(s"${KafkaConfig.EarlyStartListenersProp} contains " + + throw new ConfigException(s"${ServerConfigs.EARLY_START_LISTENERS_CONFIG} contains " + s"listener ${listenerName.value()}, but this is not contained in " + s"${SocketServerConfigs.LISTENERS_CONFIG} or ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG}") listenerName @@ -889,13 +780,13 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val failedAuthenticationDelayMs = getInt(SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG) /***************** rack configuration **************/ - val rack = Option(getString(KafkaConfig.RackProp)) + val rack = Option(getString(ServerConfigs.BROKER_RACK_CONFIG)) val replicaSelectorClassName = Option(getString(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG)) /** ********* Log Configuration ***********/ val autoCreateTopicsEnable = getBoolean(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG) val numPartitions = getInt(ServerLogConfigs.NUM_PARTITIONS_CONFIG) - val logDirs = CoreUtils.parseCsvList(Option(getString(ServerLogConfigs.LOG_DIRS_CONFIG)).getOrElse(getString(ServerLogConfigs.LOG_DIR_CONFIG))) + val logDirs: Seq[String] = Csv.parseCsvList(Option(getString(ServerLogConfigs.LOG_DIRS_CONFIG)).getOrElse(getString(ServerLogConfigs.LOG_DIR_CONFIG))).asScala def logSegmentBytes = getInt(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG) def logFlushIntervalMessages = getLong(ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG) val logCleanerThreads = getInt(CleanerConfig.LOG_CLEANER_THREADS_PROP) @@ -975,6 +866,8 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami def logMessageDownConversionEnable: Boolean = getBoolean(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG) + def logDirFailureTimeoutMs: Long = getLong(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG) + /** ********* Replication configuration ***********/ val controllerSocketTimeoutMs: Int = getInt(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG) val defaultReplicationFactor: Int = getInt(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG) @@ -1020,9 +913,9 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami } /** ********* Controlled shutdown configuration ***********/ - val controlledShutdownMaxRetries = getInt(KafkaConfig.ControlledShutdownMaxRetriesProp) - val controlledShutdownRetryBackoffMs = getLong(KafkaConfig.ControlledShutdownRetryBackoffMsProp) - val controlledShutdownEnable = getBoolean(KafkaConfig.ControlledShutdownEnableProp) + val controlledShutdownMaxRetries = getInt(ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG) + val controlledShutdownRetryBackoffMs = getLong(ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG) + val controlledShutdownEnable = getBoolean(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG) /** ********* Feature configuration ***********/ def isFeatureVersioningSupported = interBrokerProtocolVersion.isFeatureVersioningSupported @@ -1041,8 +934,11 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami throw new ConfigException(s"Disabling the '${GroupType.CLASSIC}' protocol is not supported.") } if (protocols.contains(GroupType.CONSUMER)) { + if (processRoles.isEmpty) { + throw new ConfigException(s"The new '${GroupType.CONSUMER}' rebalance protocol is only supported in KRaft cluster.") + } warn(s"The new '${GroupType.CONSUMER}' rebalance protocol is enabled along with the new group coordinator. " + - "This is part of the early access of KIP-848 and MUST NOT be used in production.") + "This is part of the preview of KIP-848 and MUST NOT be used in production.") } protocols } @@ -1060,7 +956,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val consumerGroupMinHeartbeatIntervalMs = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG) val consumerGroupMaxHeartbeatIntervalMs = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG) val consumerGroupMaxSize = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_CONFIG) - val consumerGroupAssignors = getConfiguredInstances(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, classOf[PartitionAssignor]) + val consumerGroupAssignors = getConfiguredInstances(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, classOf[ConsumerGroupPartitionAssignor]) val consumerGroupMigrationPolicy = ConsumerGroupMigrationPolicy.parse(getString(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG)) /** ********* Offset management configuration ***********/ @@ -1103,7 +999,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami // Hence the base SSL/SASL configs are not fields of KafkaConfig, listener configs should be // retrieved using KafkaConfig#valuesWithPrefixOverride private def saslEnabledMechanisms(listenerName: ListenerName): Set[String] = { - val value = valuesWithPrefixOverride(listenerName.configPrefix).get(KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG) + val value = valuesWithPrefixOverride(listenerName.configPrefix).get(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG) if (value != null) value.asInstanceOf[util.List[String]].asScala.toSet else @@ -1114,16 +1010,16 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami def interBrokerSecurityProtocol = getInterBrokerListenerNameAndSecurityProtocol._2 def controlPlaneListenerName = getControlPlaneListenerNameAndSecurityProtocol.map { case (listenerName, _) => listenerName } def controlPlaneSecurityProtocol = getControlPlaneListenerNameAndSecurityProtocol.map { case (_, securityProtocol) => securityProtocol } - def saslMechanismInterBrokerProtocol = getString(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG) + def saslMechanismInterBrokerProtocol = getString(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG) val saslInterBrokerHandshakeRequestEnable = interBrokerProtocolVersion.isSaslInterBrokerHandshakeRequestEnabled /** ********* DelegationToken Configuration **************/ - val delegationTokenSecretKey = Option(getPassword(KafkaConfig.DelegationTokenSecretKeyProp)) - .getOrElse(getPassword(KafkaConfig.DelegationTokenSecretKeyAliasProp)) + val delegationTokenSecretKey = Option(getPassword(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG)) + .getOrElse(getPassword(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG)) val tokenAuthEnabled = delegationTokenSecretKey != null && delegationTokenSecretKey.value.nonEmpty - val delegationTokenMaxLifeMs = getLong(KafkaConfig.DelegationTokenMaxLifeTimeProp) - val delegationTokenExpiryTimeMs = getLong(KafkaConfig.DelegationTokenExpiryTimeMsProp) - val delegationTokenExpiryCheckIntervalMs = getLong(KafkaConfig.DelegationTokenExpiryCheckIntervalMsProp) + val delegationTokenMaxLifeMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFETIME_CONFIG) + val delegationTokenExpiryTimeMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG) + val delegationTokenExpiryCheckIntervalMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG) /** ********* Password encryption configuration for dynamic configs *********/ def passwordEncoderSecret = Option(getPassword(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG)) @@ -1144,17 +1040,18 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val controllerQuotaWindowSizeSeconds = getInt(QuotaConfigs.CONTROLLER_QUOTA_WINDOW_SIZE_SECONDS_CONFIG) /** ********* Fetch Configuration **************/ - val maxIncrementalFetchSessionCacheSlots = getInt(KafkaConfig.MaxIncrementalFetchSessionCacheSlots) - val fetchMaxBytes = getInt(KafkaConfig.FetchMaxBytes) + val maxIncrementalFetchSessionCacheSlots = getInt(ServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_CONFIG) + val fetchMaxBytes = getInt(ServerConfigs.FETCH_MAX_BYTES_CONFIG) /** ********* Request Limit Configuration ***********/ - val maxRequestPartitionSizeLimit = getInt(KafkaConfig.MaxRequestPartitionSizeLimit) + val maxRequestPartitionSizeLimit = getInt(ServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_CONFIG) - val deleteTopicEnable = getBoolean(KafkaConfig.DeleteTopicEnableProp) - def compressionType = getString(KafkaConfig.CompressionTypeProp) - def gzipCompressionLevel = getInt(KafkaConfig.CompressionGzipLevelProp) - def lz4CompressionLevel = getInt(KafkaConfig.CompressionLz4LevelProp) - def zstdCompressionLevel = getInt(KafkaConfig.CompressionZstdLevelProp) + val deleteTopicEnable = getBoolean(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG) + def compressionType = getString(ServerConfigs.COMPRESSION_TYPE_CONFIG) + + def gzipCompressionLevel = getInt(ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG) + def lz4CompressionLevel = getInt(ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG) + def zstdCompressionLevel = getInt(ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG) /** ********* Raft Quorum Configuration *********/ val quorumVoters = getList(QuorumConfig.QUORUM_VOTERS_CONFIG) @@ -1166,8 +1063,8 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val quorumRetryBackoffMs = getInt(QuorumConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG) /** Internal Configurations **/ - val unstableApiVersionsEnabled = getBoolean(KafkaConfig.UnstableApiVersionsEnableProp) - val unstableMetadataVersionsEnabled = getBoolean(KafkaConfig.UnstableMetadataVersionsEnableProp) + val unstableApiVersionsEnabled = getBoolean(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG) + val unstableFeatureVersionsEnabled = getBoolean(ServerConfigs.UNSTABLE_FEATURE_VERSIONS_ENABLE_CONFIG) def addReconfigurable(reconfigurable: Reconfigurable): Unit = { dynamicConfig.addReconfigurable(reconfigurable) @@ -1292,7 +1189,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami // check controller listener names (they won't appear in listeners when process.roles=broker) // as well as listeners for occurrences of SSL or SASL_* if (controllerListenerNames.exists(isSslOrSasl) || - parseCsvList(getString(SocketServerConfigs.LISTENERS_CONFIG)).exists(listenerValue => isSslOrSasl(EndPoint.parseListenerName(listenerValue)))) { + Csv.parseCsvList(getString(SocketServerConfigs.LISTENERS_CONFIG)).asScala.exists(listenerValue => isSslOrSasl(EndPoint.parseListenerName(listenerValue)))) { mapValue // don't add default mappings since we found something that is SSL or SASL_* } else { // add the PLAINTEXT mappings for all controller listener names that are not explicitly PLAINTEXT @@ -1308,8 +1205,6 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami def usesTopicId: Boolean = usesSelfManagedQuorum || interBrokerProtocolVersion.isTopicIdsSupported - - val isRemoteLogStorageSystemEnabled: lang.Boolean = getBoolean(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP) def logLocalRetentionBytes: java.lang.Long = getLong(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP) def logLocalRetentionMs: java.lang.Long = getLong(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP) @@ -1319,7 +1214,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami @nowarn("cat=deprecation") private def validateValues(): Unit = { if (nodeId != brokerId) { - throw new ConfigException(s"You must set `${KRaftConfigs.NODE_ID_CONFIG}` to the same value as `${KafkaConfig.BrokerIdProp}`.") + throw new ConfigException(s"You must set `${KRaftConfigs.NODE_ID_CONFIG}` to the same value as `${ServerConfigs.BROKER_ID_CONFIG}`.") } if (requiresZookeeper) { if (zkConnect == null) { @@ -1346,9 +1241,6 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami require(logRollTimeJitterMillis >= 0, "log.roll.jitter.ms must be greater than or equal to 0") require(logRetentionTimeMillis >= 1 || logRetentionTimeMillis == -1, "log.retention.ms must be unlimited (-1) or, greater than or equal to 1") require(logDirs.nonEmpty, "At least one log directory must be defined via log.dirs or log.dir.") - if (isRemoteLogStorageSystemEnabled && logDirs.size > 1) { - throw new ConfigException(s"Multiple log directories `${logDirs.mkString(",")}` are not supported when remote log storage is enabled") - } require(logCleanerDedupeBufferSize / logCleanerThreads > 1024 * 1024, "log.cleaner.dedupe.buffer.size must be at least 1MB per cleaner thread.") require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs, "replica.socket.timeout.ms should always be at least replica.fetch.wait.max.ms" + " to prevent unnecessary socket timeouts") @@ -1517,9 +1409,9 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami require(!interBrokerUsesSasl || saslInterBrokerHandshakeRequestEnable || saslMechanismInterBrokerProtocol == SaslConfigs.GSSAPI_MECHANISM, s"Only GSSAPI mechanism is supported for inter-broker communication with SASL when inter.broker.protocol.version is set to $interBrokerProtocolVersionString") require(!interBrokerUsesSasl || saslEnabledMechanisms(interBrokerListenerName).contains(saslMechanismInterBrokerProtocol), - s"${KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG} must be included in ${KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG} when SASL is used for inter-broker communication") + s"${BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG} must be included in ${BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG} when SASL is used for inter-broker communication") require(queuedMaxBytes <= 0 || queuedMaxBytes >= socketRequestMaxBytes, - s"${KafkaConfig.QueuedMaxBytesProp} must be larger or equal to ${SocketServerConfigs.SOCKET_RECEIVE_BUFFER_BYTES_CONFIG}") + s"${ServerConfigs.QUEUED_MAX_BYTES_CONFIG} must be larger or equal to ${SocketServerConfigs.SOCKET_REQUEST_MAX_BYTES_CONFIG}") if (maxConnectionsPerIp == 0) require(maxConnectionsPerIpOverrides.nonEmpty, s"${SocketServerConfigs.MAX_CONNECTIONS_PER_IP_CONFIG} can be set to zero only if" + @@ -1535,10 +1427,10 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami s" ${SocketServerConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG}=$connectionsMaxIdleMs to prevent failed" + s" authentication responses from timing out") - val principalBuilderClass = getClass(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG) - require(principalBuilderClass != null, s"${KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG} must be non-null") + val principalBuilderClass = getClass(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG) + require(principalBuilderClass != null, s"${BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG} must be non-null") require(classOf[KafkaPrincipalSerde].isAssignableFrom(principalBuilderClass), - s"${KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG} must implement KafkaPrincipalSerde") + s"${BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG} must implement KafkaPrincipalSerde") // New group coordinator configs validation. require(consumerGroupMaxHeartbeatIntervalMs >= consumerGroupMinHeartbeatIntervalMs, diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala index d9a8a8315b764..d054d9604c71b 100755 --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -17,8 +17,8 @@ package kafka.server -import kafka.network._ -import kafka.utils._ +import kafka.network.RequestChannel +import kafka.utils.{Exit, Logging, Pool} import kafka.server.KafkaRequestHandler.{threadCurrentRequest, threadRequestChannel} import java.util.concurrent.{ConcurrentHashMap, CountDownLatch, TimeUnit} @@ -246,7 +246,7 @@ class KafkaRequestHandlerPool( } } -class BrokerTopicMetrics(name: Option[String], configOpt: java.util.Optional[KafkaConfig]) { +class BrokerTopicMetrics(name: Option[String], remoteStorageEnabled: Boolean = false) { private val metricsGroup = new KafkaMetricsGroup(this.getClass) val tags: java.util.Map[String, String] = name match { @@ -333,31 +333,30 @@ class BrokerTopicMetrics(name: Option[String], configOpt: java.util.Optional[Kaf metricTypeMap.put(BrokerTopicStats.ReassignmentBytesOutPerSec, MeterWrapper(BrokerTopicStats.ReassignmentBytesOutPerSec, "bytes")) } - configOpt.ifPresent(config => - if (config.remoteLogManagerConfig.enableRemoteStorageSystem()) { - metricTypeMap.putAll(Map( - RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName, "bytes"), - RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName, "bytes"), - RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.REMOTE_DELETE_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_DELETE_REQUESTS_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.BUILD_REMOTE_LOG_AUX_STATE_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.BUILD_REMOTE_LOG_AUX_STATE_REQUESTS_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.FAILED_REMOTE_DELETE_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_DELETE_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.FAILED_BUILD_REMOTE_LOG_AUX_STATE_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_BUILD_REMOTE_LOG_AUX_STATE_PER_SEC_METRIC.getName, "requests") - ).asJava) - - metricGaugeTypeMap.putAll(Map( - RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName), - RemoteStorageMetrics.REMOTE_COPY_LAG_SEGMENTS_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_COPY_LAG_SEGMENTS_METRIC.getName), - RemoteStorageMetrics.REMOTE_DELETE_LAG_BYTES_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_DELETE_LAG_BYTES_METRIC.getName), - RemoteStorageMetrics.REMOTE_DELETE_LAG_SEGMENTS_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_DELETE_LAG_SEGMENTS_METRIC.getName), - RemoteStorageMetrics.REMOTE_LOG_METADATA_COUNT_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_LOG_METADATA_COUNT_METRIC.getName), - RemoteStorageMetrics.REMOTE_LOG_SIZE_COMPUTATION_TIME_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_LOG_SIZE_COMPUTATION_TIME_METRIC.getName), - RemoteStorageMetrics.REMOTE_LOG_SIZE_BYTES_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_LOG_SIZE_BYTES_METRIC.getName) - ).asJava) - }) + if (remoteStorageEnabled) { + metricTypeMap.putAll(Map( + RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName, "bytes"), + RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName, "bytes"), + RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName, "requests"), + RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName, "requests"), + RemoteStorageMetrics.REMOTE_DELETE_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_DELETE_REQUESTS_PER_SEC_METRIC.getName, "requests"), + RemoteStorageMetrics.BUILD_REMOTE_LOG_AUX_STATE_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.BUILD_REMOTE_LOG_AUX_STATE_REQUESTS_PER_SEC_METRIC.getName, "requests"), + RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName, "requests"), + RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName, "requests"), + RemoteStorageMetrics.FAILED_REMOTE_DELETE_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_DELETE_PER_SEC_METRIC.getName, "requests"), + RemoteStorageMetrics.FAILED_BUILD_REMOTE_LOG_AUX_STATE_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_BUILD_REMOTE_LOG_AUX_STATE_PER_SEC_METRIC.getName, "requests") + ).asJava) + + metricGaugeTypeMap.putAll(Map( + RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName), + RemoteStorageMetrics.REMOTE_COPY_LAG_SEGMENTS_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_COPY_LAG_SEGMENTS_METRIC.getName), + RemoteStorageMetrics.REMOTE_DELETE_LAG_BYTES_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_DELETE_LAG_BYTES_METRIC.getName), + RemoteStorageMetrics.REMOTE_DELETE_LAG_SEGMENTS_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_DELETE_LAG_SEGMENTS_METRIC.getName), + RemoteStorageMetrics.REMOTE_LOG_METADATA_COUNT_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_LOG_METADATA_COUNT_METRIC.getName), + RemoteStorageMetrics.REMOTE_LOG_SIZE_COMPUTATION_TIME_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_LOG_SIZE_COMPUTATION_TIME_METRIC.getName), + RemoteStorageMetrics.REMOTE_LOG_SIZE_BYTES_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_LOG_SIZE_BYTES_METRIC.getName) + ).asJava) + } // used for testing only def metricMap: Map[String, MeterWrapper] = metricTypeMap.toMap @@ -523,11 +522,11 @@ object BrokerTopicStats { val InvalidOffsetOrSequenceRecordsPerSec = "InvalidOffsetOrSequenceRecordsPerSec" } -class BrokerTopicStats(configOpt: java.util.Optional[KafkaConfig] = java.util.Optional.empty()) extends Logging { +class BrokerTopicStats(remoteStorageEnabled: Boolean = false) extends Logging { - private val valueFactory = (k: String) => new BrokerTopicMetrics(Some(k), configOpt) + private val valueFactory = (k: String) => new BrokerTopicMetrics(Some(k), remoteStorageEnabled) private val stats = new Pool[String, BrokerTopicMetrics](Some(valueFactory)) - val allTopicsStats = new BrokerTopicMetrics(None, configOpt) + val allTopicsStats = new BrokerTopicMetrics(None, remoteStorageEnabled) def isTopicStatsExisted(topic: String): Boolean = stats.contains(topic) @@ -553,7 +552,7 @@ class BrokerTopicStats(configOpt: java.util.Optional[KafkaConfig] = java.util.Op } } - def updateReassignmentBytesOut(value: Long): Unit = { + private def updateReassignmentBytesOut(value: Long): Unit = { allTopicsStats.reassignmentBytesOutPerSec.foreach { metric => metric.mark(value) } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 8dc35ff628e41..738adab0fb0c1 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -44,7 +44,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.security.{JaasContext, JaasUtils} import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time, Utils} -import org.apache.kafka.common.{Endpoint, KafkaException, Node, TopicPartition} +import org.apache.kafka.common.{Endpoint, Node, TopicPartition} import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.image.loader.metrics.MetadataLoaderMetrics import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag @@ -165,8 +165,10 @@ class KafkaServer( var kafkaScheduler: KafkaScheduler = _ - var kraftControllerNodes: Seq[Node] = _ @volatile var metadataCache: ZkMetadataCache = _ + + @volatile var quorumControllerNodeProvider: RaftControllerNodeProvider = _ + var quotaManagers: QuotaFactory.QuotaManagers = _ val zkClientConfig: ZKClientConfig = KafkaServer.zkClientConfigFromKafkaConfig(config) @@ -274,7 +276,7 @@ class KafkaServer( createCurrentControllerIdMetric() /* register broker metrics */ - _brokerTopicStats = new BrokerTopicStats(java.util.Optional.of(config)) + _brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig.enableRemoteStorageSystem()) quotaManagers = QuotaFactory.instantiate(config, metrics, time, threadNamePrefix.getOrElse("")) KafkaBroker.notifyClusterListeners(clusterId, kafkaMetricsReporters ++ metrics.reporters.asScala) @@ -324,20 +326,13 @@ class KafkaServer( remoteLogManagerOpt = createRemoteLogManager() - if (config.migrationEnabled) { - kraftControllerNodes = QuorumConfig.voterConnectionsToNodes( - QuorumConfig.parseVoterConnections(config.quorumVoters) - ).asScala - } else { - kraftControllerNodes = Seq.empty - } metadataCache = MetadataCache.zkMetadataCache( config.brokerId, config.interBrokerProtocolVersion, brokerFeatures, - kraftControllerNodes, config.migrationEnabled) - val controllerNodeProvider = new MetadataCacheControllerNodeProvider(metadataCache, config) + val controllerNodeProvider = new MetadataCacheControllerNodeProvider(metadataCache, config, + () => Option(quorumControllerNodeProvider).map(_.getControllerInfo())) /* initialize feature change listener */ _featureChangeListener = new FinalizedFeatureChangeListener(metadataCache, _zkClient) @@ -446,8 +441,7 @@ class KafkaServer( CompletableFuture.completedFuture(quorumVoters), fatalFaultHandler = new LoggingFaultHandler("raftManager", () => shutdown()) ) - val controllerNodes = QuorumConfig.voterConnectionsToNodes(quorumVoters).asScala - val quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config, controllerNodes) + quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config) val brokerToQuorumChannelManager = new NodeToControllerChannelManagerImpl( controllerNodeProvider = quorumControllerNodeProvider, time = time, @@ -696,10 +690,6 @@ class KafkaServer( protected def createRemoteLogManager(): Option[RemoteLogManager] = { if (config.remoteLogManagerConfig.enableRemoteStorageSystem()) { - if (config.logDirs.size > 1) { - throw new KafkaException("Tiered storage is not supported with multiple log dirs.") - } - Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time, (tp: TopicPartition) => logManager.getLog(tp).asJava, (tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => { @@ -707,7 +697,7 @@ class KafkaServer( log.updateLogStartOffsetFromRemoteTier(remoteLogStartOffset) } }, - brokerTopicStats)) + brokerTopicStats, metrics)) } else { None } @@ -1076,6 +1066,8 @@ class KafkaServer( } _brokerState = BrokerState.NOT_RUNNING + quorumControllerNodeProvider = null + startupComplete.set(false) isShuttingDown.set(false) CoreUtils.swallow(AppInfoParser.unregisterAppInfo(Server.MetricsPrefix, config.brokerId.toString, metrics), this) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 015e46a76523d..2636264a57da4 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -22,7 +22,7 @@ import org.apache.kafka.admin.BrokerMetadata import org.apache.kafka.common.message.{MetadataResponseData, UpdateMetadataRequestData} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.{Cluster, Node, TopicPartition, Uuid} -import org.apache.kafka.server.common.{Features, MetadataVersion} +import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion} import java.util import scala.collection._ @@ -109,17 +109,16 @@ trait MetadataCache { def getRandomAliveBrokerId: Option[Int] - def features(): Features + def features(): FinalizedFeatures } object MetadataCache { def zkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFeatures: BrokerFeatures = BrokerFeatures.createEmpty(), - kraftControllerNodes: collection.Seq[Node] = collection.Seq.empty[Node], zkMigrationEnabled: Boolean = false) : ZkMetadataCache = { - new ZkMetadataCache(brokerId, metadataVersion, brokerFeatures, kraftControllerNodes, zkMigrationEnabled) + new ZkMetadataCache(brokerId, metadataVersion, brokerFeatures, zkMigrationEnabled) } def kRaftMetadataCache(brokerId: Int): KRaftMetadataCache = { diff --git a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala index 2190b3b3d7263..0017a5876af13 100644 --- a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala @@ -55,22 +55,15 @@ trait ControllerNodeProvider { class MetadataCacheControllerNodeProvider( val metadataCache: ZkMetadataCache, - val config: KafkaConfig + val config: KafkaConfig, + val quorumControllerNodeProvider: () => Option[ControllerInformation] ) extends ControllerNodeProvider { private val zkControllerListenerName = config.controlPlaneListenerName.getOrElse(config.interBrokerListenerName) private val zkControllerSecurityProtocol = config.controlPlaneSecurityProtocol.getOrElse(config.interBrokerSecurityProtocol) private val zkControllerSaslMechanism = config.saslMechanismInterBrokerProtocol - private val kraftControllerListenerName = if (config.controllerListenerNames.nonEmpty) - new ListenerName(config.controllerListenerNames.head) else null - private val kraftControllerSecurityProtocol = Option(kraftControllerListenerName) - .map( listener => config.effectiveListenerSecurityProtocolMap.getOrElse( - listener, SecurityProtocol.forName(kraftControllerListenerName.value()))) - .orNull - private val kraftControllerSaslMechanism = config.saslMechanismControllerProtocol - - private val emptyZkControllerInfo = ControllerInformation( + val emptyZkControllerInfo = ControllerInformation( None, zkControllerListenerName, zkControllerSecurityProtocol, @@ -85,12 +78,8 @@ class MetadataCacheControllerNodeProvider( zkControllerSecurityProtocol, zkControllerSaslMechanism, isZkController = true) - case KRaftCachedControllerId(id) => ControllerInformation( - metadataCache.getAliveBrokerNode(id, kraftControllerListenerName), - kraftControllerListenerName, - kraftControllerSecurityProtocol, - kraftControllerSaslMechanism, - isZkController = false) + case KRaftCachedControllerId(_) => + quorumControllerNodeProvider.apply().getOrElse(emptyZkControllerInfo) }.getOrElse(emptyZkControllerInfo) } } @@ -99,14 +88,12 @@ object RaftControllerNodeProvider { def apply( raftManager: RaftManager[ApiMessageAndVersion], config: KafkaConfig, - controllerQuorumVoterNodes: Seq[Node] ): RaftControllerNodeProvider = { val controllerListenerName = new ListenerName(config.controllerListenerNames.head) val controllerSecurityProtocol = config.effectiveListenerSecurityProtocolMap.getOrElse(controllerListenerName, SecurityProtocol.forName(controllerListenerName.value())) val controllerSaslMechanism = config.saslMechanismControllerProtocol new RaftControllerNodeProvider( raftManager, - controllerQuorumVoterNodes, controllerListenerName, controllerSecurityProtocol, controllerSaslMechanism @@ -120,15 +107,15 @@ object RaftControllerNodeProvider { */ class RaftControllerNodeProvider( val raftManager: RaftManager[ApiMessageAndVersion], - controllerQuorumVoterNodes: Seq[Node], val listenerName: ListenerName, val securityProtocol: SecurityProtocol, val saslMechanism: String ) extends ControllerNodeProvider with Logging { - private val idToNode = controllerQuorumVoterNodes.map(node => node.id() -> node).toMap + + private def idToNode(id: Int): Option[Node] = raftManager.voterNode(id, listenerName.value()) override def getControllerInfo(): ControllerInformation = - ControllerInformation(raftManager.leaderAndEpoch.leaderId.asScala.map(idToNode), + ControllerInformation(raftManager.leaderAndEpoch.leaderId.asScala.flatMap(idToNode), listenerName, securityProtocol, saslMechanism, isZkController = false) } diff --git a/core/src/main/scala/kafka/server/QuotaFactory.scala b/core/src/main/scala/kafka/server/QuotaFactory.scala index abee7679cddb2..f613463774af3 100644 --- a/core/src/main/scala/kafka/server/QuotaFactory.scala +++ b/core/src/main/scala/kafka/server/QuotaFactory.scala @@ -33,6 +33,8 @@ object QuotaType { case object LeaderReplication extends QuotaType case object FollowerReplication extends QuotaType case object AlterLogDirsReplication extends QuotaType + case object RLMCopy extends QuotaType + case object RLMFetch extends QuotaType def toClientQuotaType(quotaType: QuotaType): ClientQuotaType = { quotaType match { diff --git a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala index 2ea09390b3dcb..95c7a5ac3d4b1 100644 --- a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala @@ -40,7 +40,7 @@ class ReplicaAlterLogDirsThread(name: String, clientId = name, leader = leader, failedPartitions, - fetchTierStateMachine = new ReplicaAlterLogDirsTierStateMachine(), + fetchTierStateMachine = new TierStateMachine(leader, replicaMgr, true), fetchBackOffMs = fetchBackOffMs, isInterruptible = false, brokerTopicStats) { diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index c45a5d629a69b..bb073682bdfb6 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -37,7 +37,7 @@ class ReplicaFetcherThread(name: String, clientId = name, leader = leader, failedPartitions, - fetchTierStateMachine = new ReplicaFetcherTierStateMachine(leader, replicaMgr), + fetchTierStateMachine = new TierStateMachine(leader, replicaMgr, false), fetchBackOffMs = brokerConfig.replicaFetchBackoffMs, isInterruptible = false, replicaMgr.brokerTopicStats) { diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 35499430d6679..a2a070bcd0331 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -33,6 +33,7 @@ import kafka.zk.KafkaZkClient import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.DeleteRecordsResponseData.DeleteRecordsPartitionResult +import org.apache.kafka.common.message.DescribeLogDirsResponseData.DescribeLogDirsTopic import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.LeaderAndIsrResponseData.{LeaderAndIsrPartitionError, LeaderAndIsrTopicError} import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic @@ -67,7 +68,7 @@ import java.util import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.Lock import java.util.concurrent.{CompletableFuture, Future, RejectedExecutionException, TimeUnit} -import java.util.{Optional, OptionalInt, OptionalLong} +import java.util.{Collections, Optional, OptionalInt, OptionalLong} import scala.collection.{Map, Seq, Set, mutable} import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ @@ -703,6 +704,10 @@ class ReplicaManager(val config: KafkaConfig, getPartitionOrException(topicPartition).futureLog.isDefined } + def futureLogOrException(topicPartition: TopicPartition): UnifiedLog = { + getPartitionOrException(topicPartition).futureLocalLogOrException + } + def localLog(topicPartition: TopicPartition): Option[UnifiedLog] = { onlinePartition(topicPartition).flatMap(_.log) } @@ -1245,9 +1250,9 @@ class ReplicaManager(val config: KafkaConfig, val fileStore = Files.getFileStore(file) val totalBytes = adjustForLargeFileSystems(fileStore.getTotalSpace) val usableBytes = adjustForLargeFileSystems(fileStore.getUsableSpace) - logsByDir.get(absolutePath) match { + val topicInfos = logsByDir.get(absolutePath) match { case Some(logs) => - val topicInfos = logs.groupBy(_.topicPartition.topic).map{case (topic, logs) => + logs.groupBy(_.topicPartition.topic).map { case (topic, logs) => new DescribeLogDirsResponseData.DescribeLogDirsTopic().setName(topic).setPartitions( logs.filter { log => partitions.contains(log.topicPartition) @@ -1258,17 +1263,19 @@ class ReplicaManager(val config: KafkaConfig, .setOffsetLag(getLogEndOffsetLag(log.topicPartition, log.logEndOffset, log.isFuture)) .setIsFutureKey(log.isFuture) }.toList.asJava) - }.toList.asJava - - new DescribeLogDirsResponseData.DescribeLogDirsResult().setLogDir(absolutePath) - .setErrorCode(Errors.NONE.code).setTopics(topicInfos) - .setTotalBytes(totalBytes).setUsableBytes(usableBytes) + }.filterNot(_.partitions().isEmpty).toList.asJava case None => - new DescribeLogDirsResponseData.DescribeLogDirsResult().setLogDir(absolutePath) - .setErrorCode(Errors.NONE.code) - .setTotalBytes(totalBytes).setUsableBytes(usableBytes) + Collections.emptyList[DescribeLogDirsTopic]() } + val describeLogDirsResult = new DescribeLogDirsResponseData.DescribeLogDirsResult() + .setLogDir(absolutePath).setTopics(topicInfos) + .setErrorCode(Errors.NONE.code) + .setTotalBytes(totalBytes).setUsableBytes(usableBytes) + if (!topicInfos.isEmpty) + describeLogDirsResult.setTopics(topicInfos) + describeLogDirsResult + } catch { case e: KafkaStorageException => warn("Unable to describe replica dirs for %s".format(absolutePath), e) @@ -1744,8 +1751,8 @@ class ReplicaManager(val config: KafkaConfig, val leaderLogStartOffset = log.logStartOffset val leaderLogEndOffset = log.logEndOffset - if (params.isFromFollower) { - // If it is from a follower then send the offset metadata only as the data is already available in remote + if (params.isFromFollower || params.isFromFuture) { + // If it is from a follower or from a future replica, then send the offset metadata only as the data is already available in remote // storage and throw an error saying that this offset is moved to tiered storage. createLogReadResult(highWatermark, leaderLogStartOffset, leaderLogEndOffset, new OffsetMovedToTieredStorageException("Given offset" + offset + " is moved to tiered storage")) @@ -2114,16 +2121,12 @@ class ReplicaManager(val config: KafkaConfig, partition.log.foreach { _ => val leader = BrokerEndPoint(config.brokerId, "localhost", -1) - // Add future replica log to partition's map - partition.createLogIfNotExists( - isNew = false, - isFutureReplica = true, - offsetCheckpoints, - topicIds(partition.topic)) - - // pause cleaning for partitions that are being moved and start ReplicaAlterDirThread to move - // replica from source dir to destination dir - logManager.abortAndPauseCleaning(topicPartition) + // Add future replica log to partition's map if it's not existed + if (partition.maybeCreateFutureReplica(futureLog.parentDir, offsetCheckpoints, topicIds(partition.topic))) { + // pause cleaning for partitions that are being moved and start ReplicaAlterDirThread to move + // replica from source dir to destination dir + logManager.abortAndPauseCleaning(topicPartition) + } futureReplicasAndInitialOffset.put(topicPartition, InitialFetchState(topicIds(topicPartition.topic), leader, partition.getLeaderEpoch, futureLog.highWatermark)) @@ -2131,8 +2134,11 @@ class ReplicaManager(val config: KafkaConfig, } } - if (futureReplicasAndInitialOffset.nonEmpty) + if (futureReplicasAndInitialOffset.nonEmpty) { + // Even though it's possible that there is another thread adding fetcher for this future log partition, + // but it's fine because `BrokerIdAndFetcherId` will be identical and the operation will be no-op. replicaAlterLogDirsManager.addFetcherForPartitions(futureReplicasAndInitialOffset) + } } /* @@ -2440,7 +2446,9 @@ class ReplicaManager(val config: KafkaConfig, def handleLogDirFailure(dir: String, notifyController: Boolean = true): Unit = { if (!logManager.isLogDirOnline(dir)) return - warn(s"Stopping serving replicas in dir $dir") + // retrieve the UUID here because logManager.handleLogDirFailure handler removes it + val uuid = logManager.directoryId(dir) + warn(s"Stopping serving replicas in dir $dir with uuid $uuid because the log directory has failed.") replicaStateChangeLock synchronized { val newOfflinePartitions = onlinePartitionsIterator.filter { partition => partition.log.exists { _.parentDir == dir } @@ -2465,8 +2473,6 @@ class ReplicaManager(val config: KafkaConfig, warn(s"Broker $localBrokerId stopped fetcher for partitions ${newOfflinePartitions.mkString(",")} and stopped moving logs " + s"for partitions ${partitionsWithOfflineFutureReplica.mkString(",")} because they are in the failed log directory $dir.") } - // retrieve the UUID here because logManager.handleLogDirFailure handler removes it - val uuid = logManager.directoryId(dir) logManager.handleLogDirFailure(dir) if (dir == new File(config.metadataLogDir).getAbsolutePath && (config.processRoles.nonEmpty || config.migrationEnabled)) { fatal(s"Shutdown broker because the metadata log dir $dir has failed") diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 048a665757b74..ee7bfa2157ee7 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -29,7 +29,6 @@ import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.image.loader.LoaderManifest import org.apache.kafka.image.publisher.MetadataPublisher import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta} -import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.fault.FaultHandler import java.util.concurrent.CompletableFuture @@ -129,21 +128,6 @@ class BrokerMetadataPublisher( debug(s"Publishing metadata at offset $highestOffsetAndEpoch with $metadataVersionLogMsg.") } - Option(delta.featuresDelta()).foreach { featuresDelta => - featuresDelta.metadataVersionChange().ifPresent{ metadataVersion => - info(s"Updating metadata.version to ${metadataVersion.featureLevel()} at offset $highestOffsetAndEpoch.") - val currentMetadataVersion = delta.image().features().metadataVersion() - if (currentMetadataVersion.isLessThan(MetadataVersion.IBP_3_7_IV2) && metadataVersion.isAtLeast(MetadataVersion.IBP_3_7_IV2)) { - info( - s"""Resending BrokerRegistration with existing incarnation-id to inform the - |controller about log directories in the broker following metadata update: - |previousMetadataVersion: ${delta.image().features().metadataVersion()} - |newMetadataVersion: $metadataVersion""".stripMargin.linesIterator.mkString(" ").trim) - brokerLifecycleManager.handleKraftJBODMetadataVersionUpdate() - } - } - } - // Apply topic deltas. Option(delta.topicsDelta()).foreach { topicsDelta => try { diff --git a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala index f3c3512b73559..a058c435e5766 100644 --- a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala +++ b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.MetadataResponse import org.apache.kafka.image.MetadataImage import org.apache.kafka.metadata.{BrokerRegistration, PartitionRegistration, Replicas} -import org.apache.kafka.server.common.{Features, MetadataVersion} +import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion} import java.util import java.util.concurrent.ThreadLocalRandom @@ -539,9 +539,9 @@ class KRaftMetadataCache(val brokerId: Int) extends MetadataCache with Logging w override def metadataVersion(): MetadataVersion = _currentImage.features().metadataVersion() - override def features(): Features = { + override def features(): FinalizedFeatures = { val image = _currentImage - new Features(image.features().metadataVersion(), + new FinalizedFeatures(image.features().metadataVersion(), image.features().finalizedVersions(), image.highestOffsetAndEpoch().offset, true) diff --git a/core/src/main/scala/kafka/server/metadata/ZkMetadataCache.scala b/core/src/main/scala/kafka/server/metadata/ZkMetadataCache.scala index d13f9e0b7f141..20de08d24015e 100755 --- a/core/src/main/scala/kafka/server/metadata/ZkMetadataCache.scala +++ b/core/src/main/scala/kafka/server/metadata/ZkMetadataCache.scala @@ -40,7 +40,7 @@ import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AbstractControlRequest, ApiVersionsResponse, MetadataResponse, UpdateMetadataRequest} import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.server.common.{Features, MetadataVersion} +import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion} import java.util.concurrent.{ThreadLocalRandom, TimeUnit} import scala.concurrent.TimeoutException @@ -53,7 +53,7 @@ class FeatureCacheUpdateException(message: String) extends RuntimeException(mess trait ZkFinalizedFeatureCache { def waitUntilFeatureEpochOrThrow(minExpectedEpoch: Long, timeoutMs: Long): Unit - def getFeatureOption: Option[Features] + def getFeatureOption: Option[FinalizedFeatures] } case class MetadataSnapshot(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], @@ -158,7 +158,6 @@ class ZkMetadataCache( brokerId: Int, metadataVersion: MetadataVersion, brokerFeatures: BrokerFeatures, - kraftControllerNodes: Seq[Node] = Seq.empty, zkMigrationEnabled: Boolean = false) extends MetadataCache with ZkFinalizedFeatureCache with Logging { @@ -178,12 +177,10 @@ class ZkMetadataCache( private val stateChangeLogger = new StateChangeLogger(brokerId, inControllerContext = false, None) // Features are updated via ZK notification (see FinalizedFeatureChangeListener) - @volatile private var _features: Option[Features] = Option.empty + @volatile private var _features: Option[FinalizedFeatures] = Option.empty private val featureLock = new ReentrantLock() private val featureCond = featureLock.newCondition() - private val kraftControllerNodeMap = kraftControllerNodes.map(node => node.id() -> node).toMap - // This method is the main hotspot when it comes to the performance of metadata requests, // we should be careful about adding additional logic here. Relatedly, `brokers` is // `List[Integer]` instead of `List[Int]` to avoid a collection copy. @@ -350,11 +347,7 @@ class ZkMetadataCache( override def getAliveBrokerNode(brokerId: Int, listenerName: ListenerName): Option[Node] = { val snapshot = metadataSnapshot - brokerId match { - case id if snapshot.controllerId.filter(_.isInstanceOf[KRaftCachedControllerId]).exists(_.id == id) => - kraftControllerNodeMap.get(id) - case _ => snapshot.aliveBrokers.get(brokerId).flatMap(_.getNode(listenerName)) - } + snapshot.aliveBrokers.get(brokerId).flatMap(_.getNode(listenerName)) } override def getAliveBrokerNodes(listenerName: ListenerName): Iterable[Node] = { @@ -624,9 +617,9 @@ class ZkMetadataCache( override def metadataVersion(): MetadataVersion = metadataVersion - override def features(): Features = _features match { + override def features(): FinalizedFeatures = _features match { case Some(features) => features - case None => new Features(metadataVersion, + case None => new FinalizedFeatures(metadataVersion, Collections.emptyMap(), ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH, false) @@ -646,7 +639,7 @@ class ZkMetadataCache( * not modified. */ def updateFeaturesOrThrow(latestFeatures: Map[String, Short], latestEpoch: Long): Unit = { - val latest = new Features(metadataVersion, + val latest = new FinalizedFeatures(metadataVersion, latestFeatures.map(kv => (kv._1, kv._2.asInstanceOf[java.lang.Short])).asJava, latestEpoch, false) @@ -718,5 +711,5 @@ class ZkMetadataCache( } } - override def getFeatureOption: Option[Features] = _features + override def getFeatureOption: Option[FinalizedFeatures] = _features } diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index a3f97d5fb9e5a..a4777f98541d3 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -23,9 +23,8 @@ import java.io._ import com.fasterxml.jackson.databind.node.{IntNode, JsonNodeFactory, ObjectNode, TextNode} import kafka.coordinator.transaction.TransactionLog import kafka.log._ -import kafka.serializer.Decoder -import kafka.utils._ import kafka.utils.Implicits._ +import kafka.utils.{CoreUtils, VerifiableProperties} import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.message.ConsumerProtocolAssignment import org.apache.kafka.common.message.ConsumerProtocolAssignmentJsonConverter @@ -47,6 +46,7 @@ import org.apache.kafka.metadata.bootstrap.BootstrapDirectory import org.apache.kafka.snapshot.Snapshots import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils} import org.apache.kafka.storage.internals.log.{CorruptSnapshotException, LogFileUtils, OffsetIndex, ProducerStateManager, TimeIndex, TransactionIndex} +import org.apache.kafka.tools.api.{Decoder, DefaultDecoder, IntegerDecoder, LongDecoder, StringDecoder} import java.nio.ByteBuffer import scala.jdk.CollectionConverters._ @@ -604,14 +604,14 @@ object DumpLogSegments { .ofType(classOf[java.lang.Integer]) .defaultsTo(Integer.MAX_VALUE) private val deepIterationOpt = parser.accepts("deep-iteration", "if set, uses deep instead of shallow iteration. Automatically set if print-data-log is enabled.") - private val valueDecoderOpt = parser.accepts("value-decoder-class", "if set, used to deserialize the messages. This class should implement kafka.serializer.Decoder trait. Custom jar should be available in kafka/libs directory.") + private val valueDecoderOpt = parser.accepts("value-decoder-class", "if set, used to deserialize the messages. This class should implement org.apache.kafka.tools.api.Decoder trait. Custom jar should be available in kafka/libs directory.") .withOptionalArg() .ofType(classOf[java.lang.String]) - .defaultsTo("kafka.serializer.StringDecoder") - private val keyDecoderOpt = parser.accepts("key-decoder-class", "if set, used to deserialize the keys. This class should implement kafka.serializer.Decoder trait. Custom jar should be available in kafka/libs directory.") + .defaultsTo(classOf[StringDecoder].getName) + private val keyDecoderOpt = parser.accepts("key-decoder-class", "if set, used to deserialize the keys. This class should implement org.apache.kafka.tools.api.Decoder trait. Custom jar should be available in kafka/libs directory.") .withOptionalArg() .ofType(classOf[java.lang.String]) - .defaultsTo("kafka.serializer.StringDecoder") + .defaultsTo(classOf[StringDecoder].getName) private val offsetsOpt = parser.accepts("offsets-decoder", "if set, log data will be parsed as offset data from the " + "__consumer_offsets topic.") private val transactionLogOpt = parser.accepts("transaction-log-decoder", "if set, log data will be parsed as " + @@ -628,8 +628,8 @@ object DumpLogSegments { } else if (options.has(clusterMetadataOpt)) { new ClusterMetadataLogMessageParser } else { - val valueDecoder: Decoder[_] = CoreUtils.createObject[Decoder[_]](options.valueOf(valueDecoderOpt), new VerifiableProperties) - val keyDecoder: Decoder[_] = CoreUtils.createObject[Decoder[_]](options.valueOf(keyDecoderOpt), new VerifiableProperties) + val valueDecoder = newDecoder(options.valueOf(valueDecoderOpt)) + val keyDecoder = newDecoder(options.valueOf(keyDecoderOpt)) new DecoderMessageParser(keyDecoder, valueDecoder) } @@ -651,4 +651,42 @@ object DumpLogSegments { def checkArgs(): Unit = CommandLineUtils.checkRequiredArgs(parser, options, filesOpt) } + + /* + * The kafka.serializer.Decoder is deprecated in 3.8.0. This method is used to transfer the deprecated + * decoder to the new org.apache.kafka.tools.api.Decoder. Old decoders have an input VerifiableProperties. + * Remove it in new interface since it's always empty. + */ + private[tools] def newDecoder(className: String): Decoder[_] = { + try { + CoreUtils.createObject[org.apache.kafka.tools.api.Decoder[_]](convertDeprecatedDecoderClass(className)) + } catch { + case _: Exception => + // Old decoders always have an default VerifiableProperties input, because DumpLogSegments didn't provide + // any way to pass custom configs. + val decoder = CoreUtils.createObject[kafka.serializer.Decoder[_]](className, new VerifiableProperties()) + (bytes: Array[Byte]) => decoder.fromBytes(bytes) + } + } + + /* + * Covert deprecated decoder implementation to new decoder class. + */ + private[tools] def convertDeprecatedDecoderClass(className: String): String = { + if (className == "kafka.serializer.StringDecoder") { + println("kafka.serializer.StringDecoder is deprecated. Please use org.apache.kafka.tools.api.StringDecoder instead") + classOf[StringDecoder].getName + } else if (className == "kafka.serializer.LongDecoder") { + println("kafka.serializer.LongDecoder is deprecated. Please use org.apache.kafka.tools.api.LongDecoder instead") + classOf[LongDecoder].getName + } else if (className == "kafka.serializer.IntegerDecoder") { + println("kafka.serializer.IntegerDecoder is deprecated. Please use org.apache.kafka.tools.api.IntegerDecoder instead") + classOf[IntegerDecoder].getName + } else if (className == "kafka.serializer.DefaultDecoder") { + println("kafka.serializer.DefaultDecoder is deprecated. Please use org.apache.kafka.tools.api.DefaultDecoder instead") + classOf[DefaultDecoder].getName + } else { + className + } + } } diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala index 148be404bf734..c79548761d082 100644 --- a/core/src/main/scala/kafka/tools/StorageTool.scala +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -28,7 +28,7 @@ import net.sourceforge.argparse4j.inf.Namespace import org.apache.kafka.common.Uuid import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata} -import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} +import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion} import org.apache.kafka.common.metadata.FeatureLevelRecord import org.apache.kafka.common.metadata.UserScramCredentialRecord import org.apache.kafka.common.security.scram.internals.ScramMechanism @@ -36,10 +36,10 @@ import org.apache.kafka.common.security.scram.internals.ScramFormatter import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} +import org.apache.kafka.server.common.FeatureVersion import java.util -import java.util.Base64 -import java.util.Optional +import java.util.{Base64, Collections, Optional} import scala.collection.mutable import scala.jdk.CollectionConverters._ import scala.collection.mutable.ArrayBuffer @@ -60,24 +60,31 @@ object StorageTool extends Logging { case "format" => val directories = configToLogDirectories(config.get) val clusterId = namespace.getString("cluster_id") - val metadataVersion = getMetadataVersion(namespace, - Option(config.get.originals.get(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG)).map(_.toString)) - if (!metadataVersion.isKRaftSupported) { - throw new TerseFailure(s"Must specify a valid KRaft metadata.version of at least ${MetadataVersion.IBP_3_0_IV0}.") - } - if (!metadataVersion.isProduction) { - if (config.get.unstableMetadataVersionsEnabled) { - System.out.println(s"WARNING: using pre-production metadata.version $metadataVersion.") - } else { - throw new TerseFailure(s"The metadata.version $metadataVersion is not ready for production use yet.") - } - } val metaProperties = new MetaProperties.Builder(). setVersion(MetaPropertiesVersion.V1). setClusterId(clusterId). setNodeId(config.get.nodeId). build() val metadataRecords : ArrayBuffer[ApiMessageAndVersion] = ArrayBuffer() + val specifiedFeatures: util.List[String] = namespace.getList("feature") + val releaseVersionFlagSpecified = namespace.getString("release_version") != null + if (releaseVersionFlagSpecified && specifiedFeatures != null) { + throw new TerseFailure("Both --release-version and --feature were set. Only one of the two flags can be set.") + } + val featureNamesAndLevelsMap = featureNamesAndLevels(Option(specifiedFeatures).getOrElse(Collections.emptyList).asScala.toList) + val metadataVersion = getMetadataVersion(namespace, featureNamesAndLevelsMap, + Option(config.get.originals.get(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG)).map(_.toString)) + validateMetadataVersion(metadataVersion, config) + // Get all other features, validate, and create records for them + // Use latest default for features if --release-version is not specified + generateFeatureRecords( + metadataRecords, + metadataVersion, + featureNamesAndLevelsMap, + Features.PRODUCTION_FEATURES.asScala.toList, + config.get.unstableFeatureVersionsEnabled, + releaseVersionFlagSpecified + ) getUserScramCredentialRecords(namespace).foreach(userScramCredentialRecords => { if (!metadataVersion.isScramSupported) { throw new TerseFailure(s"SCRAM is only supported in metadata.version ${MetadataVersion.IBP_3_5_IV2} or later.") @@ -86,6 +93,7 @@ object StorageTool extends Logging { metadataRecords.append(new ApiMessageAndVersion(record, 0.toShort)) } }) + val bootstrapMetadata = buildBootstrapMetadata(metadataVersion, Some(metadataRecords), "format command") val ignoreFormatted = namespace.getBoolean("ignore_formatted") if (!configToSelfManagedMode(config.get)) { @@ -109,6 +117,53 @@ object StorageTool extends Logging { } } + private def validateMetadataVersion(metadataVersion: MetadataVersion, config: Option[KafkaConfig]): Unit = { + if (!metadataVersion.isKRaftSupported) { + throw new TerseFailure(s"Must specify a valid KRaft metadata.version of at least ${MetadataVersion.IBP_3_0_IV0}.") + } + if (!metadataVersion.isProduction) { + if (config.get.unstableFeatureVersionsEnabled) { + System.out.println(s"WARNING: using pre-production metadata.version $metadataVersion.") + } else { + throw new TerseFailure(s"The metadata.version $metadataVersion is not ready for production use yet.") + } + } + } + + private[tools] def generateFeatureRecords(metadataRecords: ArrayBuffer[ApiMessageAndVersion], + metadataVersion: MetadataVersion, + specifiedFeatures: Map[String, java.lang.Short], + allFeatures: List[Features], + unstableFeatureVersionsEnabled: Boolean, + releaseVersionSpecified: Boolean): Unit = { + // If we are using --release-version, the default is based on the metadata version. + val metadataVersionForDefault = if (releaseVersionSpecified) metadataVersion else MetadataVersion.LATEST_PRODUCTION + + val allNonZeroFeaturesAndLevels: ArrayBuffer[FeatureVersion] = mutable.ArrayBuffer[FeatureVersion]() + + allFeatures.foreach { feature => + val level: java.lang.Short = specifiedFeatures.getOrElse(feature.featureName, feature.defaultValue(metadataVersionForDefault)) + // Only set feature records for levels greater than 0. 0 is assumed if there is no record. Throw an error if level < 0. + if (level != 0) { + allNonZeroFeaturesAndLevels.append(feature.fromFeatureLevel(level, unstableFeatureVersionsEnabled)) + } + } + val featuresMap = Features.featureImplsToMap(allNonZeroFeaturesAndLevels.asJava) + featuresMap.put(MetadataVersion.FEATURE_NAME, metadataVersion.featureLevel) + + try { + for (feature <- allNonZeroFeaturesAndLevels) { + // In order to validate, we need all feature versions set. + Features.validateVersion(feature, featuresMap) + metadataRecords.append(new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(feature.featureName). + setFeatureLevel(feature.featureLevel), 0.toShort)) + } + } catch { + case e: Throwable => throw new TerseFailure(e.getMessage) + } + } + def parseArguments(args: Array[String]): Namespace = { val parser = ArgumentParsers. newArgumentParser("kafka-storage", /* defaultHelp */ true, /* prefixChars */ "-", /* fromFilePrefix */ "@"). @@ -141,6 +196,9 @@ object StorageTool extends Logging { formatParser.addArgument("--release-version", "-r"). action(store()). help(s"A KRaft release version to use for the initial metadata.version. The minimum is ${MetadataVersion.IBP_3_0_IV0}, the default is ${MetadataVersion.LATEST_PRODUCTION}") + formatParser.addArgument("--feature", "-f"). + help("A feature upgrade we should perform, in feature=level format. For example: `metadata.version=5`."). + action(append()); parser.parseArgsOrFail(args) } @@ -156,6 +214,7 @@ object StorageTool extends Logging { def getMetadataVersion( namespace: Namespace, + featureNamesAndLevelsMap: Map[String, java.lang.Short], defaultVersionString: Option[String] ): MetadataVersion = { val defaultValue = defaultVersionString match { @@ -163,9 +222,19 @@ object StorageTool extends Logging { case None => MetadataVersion.LATEST_PRODUCTION } - Option(namespace.getString("release_version")) - .map(ver => MetadataVersion.fromVersionString(ver)) - .getOrElse(defaultValue) + val releaseVersionTag = Option(namespace.getString("release_version")) + val featureTag = featureNamesAndLevelsMap.get(MetadataVersion.FEATURE_NAME) + + (releaseVersionTag, featureTag) match { + case (Some(_), Some(_)) => // We should throw an error before we hit this case, but include for completeness + throw new IllegalArgumentException("Both --release_version and --feature were set. Only one of the two flags can be set.") + case (Some(version), None) => + MetadataVersion.fromVersionString(version) + case (None, Some(level)) => + MetadataVersion.fromFeatureLevel(level) + case (None, None) => + defaultValue + } } private def getUserScramCredentialRecord( @@ -253,7 +322,7 @@ object StorageTool extends Logging { .setServerKey(formatter.serverKey(saltedPassword)) .setIterations(iterations) } catch { - case e: Throwable => + case e: Throwable => throw new TerseFailure(s"Error attempting to create UserScramCredentialRecord: ${e.getMessage}") } myrecord @@ -464,9 +533,32 @@ object StorageTool extends Logging { copier.setWriteErrorHandler((logDir, e) => { throw new TerseFailure(s"Error while writing meta.properties file $logDir: ${e.getMessage}") }) - copier.writeLogDirChanges() }) + copier.writeLogDirChanges() } 0 } + + private def parseNameAndLevel(input: String): (String, java.lang.Short) = { + val equalsIndex = input.indexOf("=") + if (equalsIndex < 0) + throw new RuntimeException("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 + try { + levelString.toShort + } catch { + case _: Throwable => + throw new RuntimeException("Can't parse feature=level string " + input + ": " + "unable to parse " + levelString + " as a short.") + } + (name, levelString.toShort) + } + + def featureNamesAndLevels(features: List[String]): Map[String, java.lang.Short] = { + features.map { (feature: String) => + // Ensure the feature exists + val nameAndLevel = parseNameAndLevel(feature) + (nameAndLevel._1, nameAndLevel._2) + }.toMap + } } diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index 5060e255ecb3a..d357ad0bd5635 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -37,7 +37,7 @@ import org.apache.kafka.common.{TopicPartition, Uuid, protocol} import org.apache.kafka.raft.errors.NotLeaderException import org.apache.kafka.raft.{Batch, BatchReader, LeaderAndEpoch, RaftClient, QuorumConfig} import org.apache.kafka.security.CredentialProvider -import org.apache.kafka.server.common.{Features, MetadataVersion} +import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion} import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.server.config.KRaftConfigs import org.apache.kafka.server.fault.ProcessTerminatingFaultHandler @@ -81,7 +81,7 @@ class TestRaftServer( ListenerType.CONTROLLER, true, false, - () => Features.fromKRaftVersion(MetadataVersion.MINIMUM_KRAFT_VERSION)) + () => FinalizedFeatures.fromKRaftVersion(MetadataVersion.MINIMUM_KRAFT_VERSION)) socketServer = new SocketServer(config, metrics, time, credentialProvider, apiVersionManager) raftManager = new KafkaRaftManager[Array[Byte]]( diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index 0b445ed1a3cd5..8da7a4e7cc147 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -32,6 +32,7 @@ import org.apache.commons.validator.routines.InetAddressValidator import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.util.Csv import org.slf4j.event.Level import java.util @@ -109,17 +110,6 @@ object CoreUtils { } } - /** - * Parse a comma separated string into a sequence of strings. - * Whitespace surrounding the comma will be removed. - */ - def parseCsvList(csvList: String): Seq[String] = { - if (csvList == null || csvList.isEmpty) - Seq.empty[String] - else - csvList.split("\\s*,\\s*").filter(v => !v.equals("")) - } - /** * Create an instance of the class with the given class name */ @@ -219,8 +209,8 @@ object CoreUtils { } val endPoints = try { - val listenerList = parseCsvList(listeners) - listenerList.map(EndPoint.createEndPoint(_, Some(securityProtocolMap))) + val listenerList = Csv.parseCsvList(listeners) + listenerList.asScala.map(EndPoint.createEndPoint(_, Some(securityProtocolMap))) } catch { case e: Exception => throw new IllegalArgumentException(s"Error creating broker listeners from '$listeners': ${e.getMessage}", e) diff --git a/core/src/main/scala/kafka/zk/AdminZkClient.scala b/core/src/main/scala/kafka/zk/AdminZkClient.scala index efecfe854bbf2..604e03c7ed436 100644 --- a/core/src/main/scala/kafka/zk/AdminZkClient.scala +++ b/core/src/main/scala/kafka/zk/AdminZkClient.scala @@ -163,7 +163,7 @@ class AdminZkClient(zkClient: KafkaZkClient, LogConfig.validate(config, kafkaConfig.map(_.extractLogConfigMap).getOrElse(Collections.emptyMap()), - kafkaConfig.exists(_.isRemoteLogStorageSystemEnabled)) + kafkaConfig.exists(_.remoteLogManagerConfig.enableRemoteStorageSystem())) } private def writeTopicPartitionAssignment(topic: String, replicaAssignment: Map[Int, ReplicaAssignment], @@ -481,7 +481,7 @@ class AdminZkClient(zkClient: KafkaZkClient, // remove the topic overrides LogConfig.validate(configs, kafkaConfig.map(_.extractLogConfigMap).getOrElse(Collections.emptyMap()), - kafkaConfig.exists(_.isRemoteLogStorageSystemEnabled)) + kafkaConfig.exists(_.remoteLogManagerConfig.enableRemoteStorageSystem())) } /** diff --git a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java index ab0f30f49b660..23c250a764ec1 100644 --- a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java +++ b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java @@ -26,23 +26,22 @@ import kafka.zk.AdminZkClient; import kafka.zk.BrokerInfo; import kafka.zk.KafkaZkClient; +import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.security.PasswordEncoder; -import org.apache.kafka.security.PasswordEncoderConfigs; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.config.ZooKeeperInternals; +import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.extension.ExtendWith; -import scala.collection.JavaConverters; -import scala.collection.Seq; +import org.junit.platform.commons.util.StringUtils; import java.io.ByteArrayOutputStream; import java.io.PrintStream; -import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -50,234 +49,512 @@ import java.util.Optional; import java.util.Properties; import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.Stream; +import static java.util.Arrays.asList; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static java.util.Collections.singletonMap; +import static org.apache.kafka.security.PasswordEncoderConfigs.PASSWORD_ENCODER_CIPHER_ALGORITHM_CONFIG; +import static org.apache.kafka.security.PasswordEncoderConfigs.PASSWORD_ENCODER_ITERATIONS_CONFIG; +import static org.apache.kafka.security.PasswordEncoderConfigs.PASSWORD_ENCODER_KEYFACTORY_ALGORITHM_CONFIG; +import static org.apache.kafka.security.PasswordEncoderConfigs.PASSWORD_ENCODER_KEY_LENGTH_CONFIG; +import static org.apache.kafka.security.PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters @ExtendWith(value = ClusterTestExtensions.class) @Tag("integration") public class ConfigCommandIntegrationTest { - AdminZkClient adminZkClient; - List alterOpts; + private List alterOpts; + private final String defaultBrokerId = "0"; private final ClusterInstance cluster; + private static Runnable run(Stream command) { + return () -> { + try { + ConfigCommand.main(command.toArray(String[]::new)); + } catch (RuntimeException e) { + // do nothing. + } finally { + Exit.resetExitProcedure(); + } + }; + } + public ConfigCommandIntegrationTest(ClusterInstance cluster) { this.cluster = cluster; } - @ClusterTest(types = {Type.ZK, Type.KRAFT}) + @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}) public void testExitWithNonZeroStatusOnUpdatingUnallowedConfig() { assertNonZeroStatusExit(Stream.concat(quorumArgs(), Stream.of( "--entity-name", cluster.isKRaftTest() ? "0" : "1", "--entity-type", "brokers", "--alter", "--add-config", "security.inter.broker.protocol=PLAINTEXT")), - errOut -> - assertTrue(errOut.contains("Cannot update these configs dynamically: Set(security.inter.broker.protocol)"), errOut)); + errOut -> assertTrue(errOut.contains("Cannot update these configs dynamically: Set(security.inter.broker.protocol)"), errOut)); } - @ClusterTest(types = {Type.ZK}) public void testExitWithNonZeroStatusOnZkCommandAlterUserQuota() { assertNonZeroStatusExit(Stream.concat(quorumArgs(), Stream.of( "--entity-type", "users", "--entity-name", "admin", "--alter", "--add-config", "consumer_byte_rate=20000")), - errOut -> - assertTrue(errOut.contains("User configuration updates using ZooKeeper are only supported for SCRAM credential updates."), errOut)); + errOut -> assertTrue(errOut.contains("User configuration updates using ZooKeeper are only supported for SCRAM credential updates."), errOut)); } - public static void assertNonZeroStatusExit(Stream args, Consumer checkErrOut) { - AtomicReference exitStatus = new AtomicReference<>(); - Exit.setExitProcedure((status, __) -> { - exitStatus.set(status); - throw new RuntimeException(); - }); - - String errOut = captureStandardErr(() -> { - try { - ConfigCommand.main(args.toArray(String[]::new)); - } catch (RuntimeException e) { - // do nothing. - } finally { - Exit.resetExitProcedure(); - } - }); - - checkErrOut.accept(errOut); - assertNotNull(exitStatus.get()); - assertEquals(1, exitStatus.get()); - } - - private Stream quorumArgs() { - return cluster.isKRaftTest() - ? Stream.of("--bootstrap-server", cluster.bootstrapServers()) - : Stream.of("--zookeeper", ((ZkClusterInvocationContext.ZkClusterInstance) cluster).getUnderlying().zkConnect()); - } - - public List entityOp(Optional brokerId) { - return brokerId.map(id -> Arrays.asList("--entity-name", id)).orElse(Collections.singletonList("--entity-default")); - } - - public void alterConfigWithZk(KafkaZkClient zkClient, Map configs, Optional brokerId) throws Exception { - alterConfigWithZk(zkClient, configs, brokerId, Collections.emptyMap()); - } - - public void alterConfigWithZk(KafkaZkClient zkClient, Map configs, Optional brokerId, Map encoderConfigs) { - String configStr = Stream.of(configs.entrySet(), encoderConfigs.entrySet()) - .flatMap(Set::stream) - .map(e -> e.getKey() + "=" + e.getValue()) - .collect(Collectors.joining(",")); - ConfigCommand.ConfigCommandOptions addOpts = new ConfigCommand.ConfigCommandOptions(toArray(alterOpts, entityOp(brokerId), Arrays.asList("--add-config", configStr))); - ConfigCommand.alterConfigWithZk(zkClient, addOpts, adminZkClient); - } - - void verifyConfig(KafkaZkClient zkClient, Map configs, Optional brokerId) { - Properties entityConfigs = zkClient.getEntityConfigs("brokers", brokerId.orElse(ZooKeeperInternals.DEFAULT_STRING)); - assertEquals(configs, entityConfigs); - } - - void alterAndVerifyConfig(KafkaZkClient zkClient, Map configs, Optional brokerId) throws Exception { - alterConfigWithZk(zkClient, configs, brokerId); - verifyConfig(zkClient, configs, brokerId); - } + @ClusterTest(types = {Type.CO_KRAFT, Type.KRAFT}) + public void testNullStatusOnKraftCommandAlterUserQuota() { + Stream command = Stream.concat(quorumArgs(), Stream.of( + "--entity-type", "users", + "--entity-name", "admin", + "--alter", "--add-config", "consumer_byte_rate=20000")); + String message = captureStandardMsg(run(command)); - void deleteAndVerifyConfig(KafkaZkClient zkClient, Set configNames, Optional brokerId) { - ConfigCommand.ConfigCommandOptions deleteOpts = new ConfigCommand.ConfigCommandOptions(toArray(alterOpts, entityOp(brokerId), Arrays.asList("--delete-config", String.join(",", configNames)))); - ConfigCommand.alterConfigWithZk(zkClient, deleteOpts, adminZkClient); - verifyConfig(zkClient, Collections.emptyMap(), brokerId); + assertTrue(StringUtils.isBlank(message), message); } - @ClusterTest(types = {Type.ZK}) + @ClusterTest(types = Type.ZK) public void testDynamicBrokerConfigUpdateUsingZooKeeper() throws Exception { cluster.shutdownBroker(0); String zkConnect = ((ZkClusterInvocationContext.ZkClusterInstance) cluster).getUnderlying().zkConnect(); KafkaZkClient zkClient = ((ZkClusterInvocationContext.ZkClusterInstance) cluster).getUnderlying().zkClient(); String brokerId = "1"; - adminZkClient = new AdminZkClient(zkClient, scala.None$.empty()); - alterOpts = Arrays.asList("--zookeeper", zkConnect, "--entity-type", "brokers", "--alter"); + AdminZkClient adminZkClient = new AdminZkClient(zkClient, scala.None$.empty()); + alterOpts = asList("--zookeeper", zkConnect, "--entity-type", "brokers", "--alter"); // Add config - alterAndVerifyConfig(zkClient, Collections.singletonMap("message.max.size", "110000"), Optional.of(brokerId)); - alterAndVerifyConfig(zkClient, Collections.singletonMap("message.max.size", "120000"), Optional.empty()); + alterAndVerifyConfig(zkClient, adminZkClient, Optional.of(brokerId), + singletonMap("message.max.bytes", "110000")); + alterAndVerifyConfig(zkClient, adminZkClient, Optional.empty(), + singletonMap("message.max.bytes", "120000")); // Change config - alterAndVerifyConfig(zkClient, Collections.singletonMap("message.max.size", "130000"), Optional.of(brokerId)); - alterAndVerifyConfig(zkClient, Collections.singletonMap("message.max.size", "140000"), Optional.empty()); + alterAndVerifyConfig(zkClient, adminZkClient, Optional.of(brokerId), + singletonMap("message.max.bytes", "130000")); + alterAndVerifyConfig(zkClient, adminZkClient, Optional.empty(), + singletonMap("message.max.bytes", "140000")); // Delete config - deleteAndVerifyConfig(zkClient, Collections.singleton("message.max.size"), Optional.of(brokerId)); - deleteAndVerifyConfig(zkClient, Collections.singleton("message.max.size"), Optional.empty()); + deleteAndVerifyConfig(zkClient, adminZkClient, Optional.of(brokerId), + singleton("message.max.bytes")); + deleteAndVerifyConfig(zkClient, adminZkClient, Optional.empty(), + singleton("message.max.bytes")); // Listener configs: should work only with listener name - alterAndVerifyConfig(zkClient, Collections.singletonMap("listener.name.external.ssl.keystore.location", "/tmp/test.jks"), Optional.of(brokerId)); + alterAndVerifyConfig(zkClient, adminZkClient, Optional.of(brokerId), + singletonMap("listener.name.internal.ssl.keystore.location", "/tmp/test.jks")); assertThrows(ConfigException.class, - () -> alterConfigWithZk(zkClient, Collections.singletonMap("ssl.keystore.location", "/tmp/test.jks"), Optional.of(brokerId))); + () -> alterConfigWithZk(zkClient, adminZkClient, Optional.of(brokerId), + singletonMap("ssl.keystore.location", "/tmp/test.jks"))); // Per-broker config configured at default cluster-level should fail assertThrows(ConfigException.class, - () -> alterConfigWithZk(zkClient, Collections.singletonMap("listener.name.external.ssl.keystore.location", "/tmp/test.jks"), Optional.empty())); - deleteAndVerifyConfig(zkClient, Collections.singleton("listener.name.external.ssl.keystore.location"), Optional.of(brokerId)); + () -> alterConfigWithZk(zkClient, adminZkClient, Optional.empty(), + singletonMap("listener.name.internal.ssl.keystore.location", "/tmp/test.jks"))); + deleteAndVerifyConfig(zkClient, adminZkClient, Optional.of(brokerId), + singleton("listener.name.internal.ssl.keystore.location")); // Password config update without encoder secret should fail assertThrows(IllegalArgumentException.class, - () -> alterConfigWithZk(zkClient, Collections.singletonMap("listener.name.external.ssl.keystore.password", "secret"), Optional.of(brokerId))); + () -> alterConfigWithZk(zkClient, adminZkClient, Optional.of(brokerId), + singletonMap("listener.name.external.ssl.keystore.password", "secret"))); // Password config update with encoder secret should succeed and encoded password must be stored in ZK Map configs = new HashMap<>(); configs.put("listener.name.external.ssl.keystore.password", "secret"); configs.put("log.cleaner.threads", "2"); - Map encoderConfigs = Collections.singletonMap(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, "encoder-secret"); - alterConfigWithZk(zkClient, configs, Optional.of(brokerId), encoderConfigs); + Map encoderConfigs = new HashMap<>(configs); + encoderConfigs.put(PASSWORD_ENCODER_SECRET_CONFIG, "encoder-secret"); + alterConfigWithZk(zkClient, adminZkClient, Optional.of(brokerId), encoderConfigs); Properties brokerConfigs = zkClient.getEntityConfigs("brokers", brokerId); - assertFalse(brokerConfigs.contains(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG), "Encoder secret stored in ZooKeeper"); + assertFalse(brokerConfigs.contains(PASSWORD_ENCODER_SECRET_CONFIG), "Encoder secret stored in ZooKeeper"); assertEquals("2", brokerConfigs.getProperty("log.cleaner.threads")); // not encoded String encodedPassword = brokerConfigs.getProperty("listener.name.external.ssl.keystore.password"); - PasswordEncoder passwordEncoder = ConfigCommand.createPasswordEncoder(JavaConverters.mapAsScalaMap(encoderConfigs)); + PasswordEncoder passwordEncoder = ConfigCommand.createPasswordEncoder(encoderConfigs); assertEquals("secret", passwordEncoder.decode(encodedPassword).value()); assertEquals(configs.size(), brokerConfigs.size()); // Password config update with overrides for encoder parameters - Map configs2 = Collections.singletonMap("listener.name.internal.ssl.keystore.password", "secret2"); - Map encoderConfigs2 = new HashMap<>(); - encoderConfigs2.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, "encoder-secret"); - encoderConfigs2.put(PasswordEncoderConfigs.PASSWORD_ENCODER_CIPHER_ALGORITHM_CONFIG, "DES/CBC/PKCS5Padding"); - encoderConfigs2.put(PasswordEncoderConfigs.PASSWORD_ENCODER_ITERATIONS_CONFIG, "1024"); - encoderConfigs2.put(PasswordEncoderConfigs.PASSWORD_ENCODER_KEYFACTORY_ALGORITHM_CONFIG, "PBKDF2WithHmacSHA1"); - encoderConfigs2.put(PasswordEncoderConfigs.PASSWORD_ENCODER_KEY_LENGTH_CONFIG, "64"); - alterConfigWithZk(zkClient, configs2, Optional.of(brokerId), encoderConfigs2); + Map encoderConfigs2 = generateEncodeConfig(); + alterConfigWithZk(zkClient, adminZkClient, Optional.of(brokerId), encoderConfigs2); Properties brokerConfigs2 = zkClient.getEntityConfigs("brokers", brokerId); - String encodedPassword2 = brokerConfigs2.getProperty("listener.name.internal.ssl.keystore.password"); - assertEquals("secret2", ConfigCommand.createPasswordEncoder(JavaConverters.mapAsScalaMap(encoderConfigs)).decode(encodedPassword2).value()); - assertEquals("secret2", ConfigCommand.createPasswordEncoder(JavaConverters.mapAsScalaMap(encoderConfigs2)).decode(encodedPassword2).value()); + String encodedPassword2 = brokerConfigs2.getProperty("listener.name.external.ssl.keystore.password"); + assertEquals("secret2", ConfigCommand.createPasswordEncoder(encoderConfigs) + .decode(encodedPassword2).value()); + assertEquals("secret2", ConfigCommand.createPasswordEncoder(encoderConfigs2) + .decode(encodedPassword2).value()); // Password config update at default cluster-level should fail - assertThrows(ConfigException.class, () -> alterConfigWithZk(zkClient, configs, Optional.empty(), encoderConfigs)); + assertThrows(ConfigException.class, + () -> alterConfigWithZk(zkClient, adminZkClient, Optional.empty(), encoderConfigs)); // Dynamic config updates using ZK should fail if broker is running. registerBrokerInZk(zkClient, Integer.parseInt(brokerId)); - assertThrows(IllegalArgumentException.class, () -> alterConfigWithZk(zkClient, Collections.singletonMap("message.max.size", "210000"), Optional.of(brokerId))); - assertThrows(IllegalArgumentException.class, () -> alterConfigWithZk(zkClient, Collections.singletonMap("message.max.size", "220000"), Optional.empty())); + assertThrows(IllegalArgumentException.class, + () -> alterConfigWithZk(zkClient, adminZkClient, + Optional.of(brokerId), singletonMap("message.max.bytes", "210000"))); + assertThrows(IllegalArgumentException.class, + () -> alterConfigWithZk(zkClient, adminZkClient, + Optional.empty(), singletonMap("message.max.bytes", "220000"))); // Dynamic config updates using ZK should for a different broker that is not running should succeed - alterAndVerifyConfig(zkClient, Collections.singletonMap("message.max.size", "230000"), Optional.of("2")); + alterAndVerifyConfig(zkClient, adminZkClient, Optional.of("2"), singletonMap("message.max.bytes", "230000")); + } + + @ClusterTest(types = {Type.CO_KRAFT, Type.KRAFT}) + public void testDynamicBrokerConfigUpdateUsingKraft() throws Exception { + alterOpts = generateDefaultAlterOpts(cluster.bootstrapServers()); + + try (Admin client = cluster.createAdminClient()) { + // Add config + alterAndVerifyConfig(client, Optional.of(defaultBrokerId), singletonMap("message.max.bytes", "110000")); + alterAndVerifyConfig(client, Optional.empty(), singletonMap("message.max.bytes", "120000")); + + // Change config + alterAndVerifyConfig(client, Optional.of(defaultBrokerId), singletonMap("message.max.bytes", "130000")); + alterAndVerifyConfig(client, Optional.empty(), singletonMap("message.max.bytes", "140000")); + + // Delete config + deleteAndVerifyConfig(client, Optional.of(defaultBrokerId), singleton("message.max.bytes")); + + // Listener configs: should work only with listener name + alterAndVerifyConfig(client, Optional.of(defaultBrokerId), + singletonMap("listener.name.internal.ssl.keystore.location", "/tmp/test.jks")); + alterConfigWithKraft(client, Optional.empty(), + singletonMap("listener.name.internal.ssl.keystore.location", "/tmp/test.jks")); + deleteAndVerifyConfig(client, Optional.of(defaultBrokerId), + singleton("listener.name.internal.ssl.keystore.location")); + alterConfigWithKraft(client, Optional.of(defaultBrokerId), + singletonMap("listener.name.external.ssl.keystore.password", "secret")); + + // Password config update with encoder secret should succeed and encoded password must be stored in ZK + Map configs = new HashMap<>(); + configs.put("listener.name.external.ssl.keystore.password", "secret"); + configs.put("log.cleaner.threads", "2"); + // Password encoder configs + configs.put(PASSWORD_ENCODER_SECRET_CONFIG, "encoder-secret"); + + // Password config update at default cluster-level should fail + assertThrows(ExecutionException.class, + () -> alterConfigWithKraft(client, Optional.of(defaultBrokerId), configs)); + } + } + + @ClusterTest(types = {Type.ZK}) + public void testAlterReadOnlyConfigInZookeeperThenShouldFail() { + cluster.shutdownBroker(0); + String zkConnect = ((ZkClusterInvocationContext.ZkClusterInstance) cluster).getUnderlying().zkConnect(); + KafkaZkClient zkClient = ((ZkClusterInvocationContext.ZkClusterInstance) cluster).getUnderlying().zkClient(); + AdminZkClient adminZkClient = new AdminZkClient(zkClient, scala.None$.empty()); + alterOpts = generateDefaultAlterOpts(zkConnect); + + assertThrows(ConfigException.class, + () -> alterConfigWithZk(zkClient, adminZkClient, Optional.of(defaultBrokerId), + singletonMap("auto.create.topics.enable", "false"))); + assertThrows(ConfigException.class, + () -> alterConfigWithZk(zkClient, adminZkClient, Optional.of(defaultBrokerId), + singletonMap("auto.leader.rebalance.enable", "false"))); + assertThrows(ConfigException.class, + () -> alterConfigWithZk(zkClient, adminZkClient, Optional.of(defaultBrokerId), + singletonMap("broker.id", "1"))); + } + + @ClusterTest(types = {Type.CO_KRAFT, Type.KRAFT}) + public void testAlterReadOnlyConfigInKRaftThenShouldFail() { + alterOpts = generateDefaultAlterOpts(cluster.bootstrapServers()); + + try (Admin client = cluster.createAdminClient()) { + assertThrows(ExecutionException.class, + () -> alterConfigWithKraft(client, Optional.of(defaultBrokerId), + singletonMap("auto.create.topics.enable", "false"))); + assertThrows(ExecutionException.class, + () -> alterConfigWithKraft(client, Optional.of(defaultBrokerId), + singletonMap("auto.leader.rebalance.enable", "false"))); + assertThrows(ExecutionException.class, + () -> alterConfigWithKraft(client, Optional.of(defaultBrokerId), + singletonMap("broker.id", "1"))); + } + } + + @ClusterTest(types = {Type.ZK}) + public void testUpdateClusterWideConfigInZookeeperThenShouldSuccessful() { + cluster.shutdownBroker(0); + String zkConnect = ((ZkClusterInvocationContext.ZkClusterInstance) cluster).getUnderlying().zkConnect(); + KafkaZkClient zkClient = ((ZkClusterInvocationContext.ZkClusterInstance) cluster).getUnderlying().zkClient(); + AdminZkClient adminZkClient = new AdminZkClient(zkClient, scala.None$.empty()); + alterOpts = generateDefaultAlterOpts(zkConnect); + + Map configs = new HashMap<>(); + configs.put("log.flush.interval.messages", "100"); + configs.put("log.retention.bytes", "20"); + configs.put("log.retention.ms", "2"); + + alterAndVerifyConfig(zkClient, adminZkClient, Optional.of(defaultBrokerId), configs); + } + + @ClusterTest(types = {Type.CO_KRAFT, Type.KRAFT}) + public void testUpdateClusterWideConfigInKRaftThenShouldSuccessful() throws Exception { + alterOpts = generateDefaultAlterOpts(cluster.bootstrapServers()); + + try (Admin client = cluster.createAdminClient()) { + alterAndVerifyConfig(client, Optional.of(defaultBrokerId), + singletonMap("log.flush.interval.messages", "100")); + alterAndVerifyConfig(client, Optional.of(defaultBrokerId), + singletonMap("log.retention.bytes", "20")); + alterAndVerifyConfig(client, Optional.of(defaultBrokerId), + singletonMap("log.retention.ms", "2")); + } + } + + @ClusterTest(types = {Type.ZK}) + public void testUpdatePerBrokerConfigWithListenerNameInZookeeperThenShouldSuccessful() { + cluster.shutdownBroker(0); + String zkConnect = ((ZkClusterInvocationContext.ZkClusterInstance) cluster).getUnderlying().zkConnect(); + KafkaZkClient zkClient = ((ZkClusterInvocationContext.ZkClusterInstance) cluster).getUnderlying().zkClient(); + AdminZkClient adminZkClient = new AdminZkClient(zkClient, scala.None$.empty()); + alterOpts = generateDefaultAlterOpts(zkConnect); + + String listenerName = "listener.name.internal."; + String sslTruststoreType = listenerName + "ssl.truststore.type"; + String sslTruststoreLocation = listenerName + "ssl.truststore.location"; + String sslTruststorePassword = listenerName + "ssl.truststore.password"; + + Map configs = new HashMap<>(); + configs.put(sslTruststoreType, "PKCS12"); + configs.put(sslTruststoreLocation, "/temp/test.jks"); + configs.put("password.encoder.secret", "encoder-secret"); + configs.put(sslTruststorePassword, "password"); + + alterConfigWithZk(zkClient, adminZkClient, Optional.of(defaultBrokerId), configs); + + Properties properties = zkClient.getEntityConfigs("brokers", defaultBrokerId); + assertTrue(properties.containsKey(sslTruststorePassword)); + assertEquals(configs.get(sslTruststoreType), properties.getProperty(sslTruststoreType)); + assertEquals(configs.get(sslTruststoreLocation), properties.getProperty(sslTruststoreLocation)); + } + + @ClusterTest(types = {Type.CO_KRAFT, Type.KRAFT}) + public void testUpdatePerBrokerConfigWithListenerNameInKRaftThenShouldSuccessful() throws Exception { + alterOpts = generateDefaultAlterOpts(cluster.bootstrapServers()); + String listenerName = "listener.name.internal."; + + try (Admin client = cluster.createAdminClient()) { + alterAndVerifyConfig(client, Optional.of(defaultBrokerId), + singletonMap(listenerName + "ssl.truststore.type", "PKCS12")); + alterAndVerifyConfig(client, Optional.of(defaultBrokerId), + singletonMap(listenerName + "ssl.truststore.location", "/temp/test.jks")); + + alterConfigWithKraft(client, Optional.of(defaultBrokerId), + singletonMap(listenerName + "ssl.truststore.password", "password")); + verifyConfigDefaultValue(client, Optional.of(defaultBrokerId), + singleton(listenerName + "ssl.truststore.password")); + } + } + + @ClusterTest(types = {Type.ZK}) + public void testUpdatePerBrokerConfigInZookeeperThenShouldFail() { + cluster.shutdownBroker(0); + String zkConnect = ((ZkClusterInvocationContext.ZkClusterInstance) cluster).getUnderlying().zkConnect(); + KafkaZkClient zkClient = ((ZkClusterInvocationContext.ZkClusterInstance) cluster).getUnderlying().zkClient(); + AdminZkClient adminZkClient = new AdminZkClient(zkClient, scala.None$.empty()); + alterOpts = generateDefaultAlterOpts(zkConnect); + + assertThrows(ConfigException.class, () -> + alterAndVerifyConfig(zkClient, adminZkClient, Optional.of(defaultBrokerId), + singletonMap("ssl.truststore.type", "PKCS12"))); + assertThrows(ConfigException.class, () -> + alterAndVerifyConfig(zkClient, adminZkClient, Optional.of(defaultBrokerId), + singletonMap("ssl.truststore.location", "/temp/test.jks"))); + assertThrows(ConfigException.class, () -> + alterAndVerifyConfig(zkClient, adminZkClient, Optional.of(defaultBrokerId), + singletonMap("ssl.truststore.password", "password"))); + } + + @ClusterTest(types = {Type.CO_KRAFT, Type.KRAFT}) + public void testUpdatePerBrokerConfigInKRaftThenShouldFail() { + alterOpts = generateDefaultAlterOpts(cluster.bootstrapServers()); + + try (Admin client = cluster.createAdminClient()) { + assertThrows(ExecutionException.class, + () -> alterConfigWithKraft(client, Optional.of(defaultBrokerId), + singletonMap("ssl.truststore.type", "PKCS12"))); + assertThrows(ExecutionException.class, + () -> alterConfigWithKraft(client, Optional.of(defaultBrokerId), + singletonMap("ssl.truststore.location", "/temp/test.jks"))); + assertThrows(ExecutionException.class, + () -> alterConfigWithKraft(client, Optional.of(defaultBrokerId), + singletonMap("ssl.truststore.password", "password"))); + } + } + + private void assertNonZeroStatusExit(Stream args, Consumer checkErrOut) { + AtomicReference exitStatus = new AtomicReference<>(); + Exit.setExitProcedure((status, __) -> { + exitStatus.set(status); + throw new RuntimeException(); + }); + + String errOut = captureStandardMsg(run(args)); + + checkErrOut.accept(errOut); + assertNotNull(exitStatus.get()); + assertEquals(1, exitStatus.get()); + } + + private Stream quorumArgs() { + return cluster.isKRaftTest() + ? Stream.of("--bootstrap-server", cluster.bootstrapServers()) + : Stream.of("--zookeeper", ((ZkClusterInvocationContext.ZkClusterInstance) cluster).getUnderlying().zkConnect()); + } + + private void verifyConfig(KafkaZkClient zkClient, Optional brokerId, Map config) { + Properties entityConfigs = zkClient.getEntityConfigs("brokers", + brokerId.orElse(ZooKeeperInternals.DEFAULT_STRING)); + assertEquals(config, entityConfigs); + } + + private void alterAndVerifyConfig(KafkaZkClient zkClient, AdminZkClient adminZkClient, + Optional brokerId, Map configs) { + alterConfigWithZk(zkClient, adminZkClient, brokerId, configs); + verifyConfig(zkClient, brokerId, configs); + } + + private void alterConfigWithZk(KafkaZkClient zkClient, AdminZkClient adminZkClient, + Optional brokerId, Map config) { + String configStr = transferConfigMapToString(config); + ConfigCommand.ConfigCommandOptions addOpts = + new ConfigCommand.ConfigCommandOptions(toArray(alterOpts, entityOp(brokerId), asList("--add-config", configStr))); + ConfigCommand.alterConfigWithZk(zkClient, addOpts, adminZkClient); + } + + private List entityOp(Optional brokerId) { + return brokerId.map(id -> asList("--entity-name", id)) + .orElse(singletonList("--entity-default")); + } + + private void deleteAndVerifyConfig(KafkaZkClient zkClient, AdminZkClient adminZkClient, + Optional brokerId, Set configNames) { + ConfigCommand.ConfigCommandOptions deleteOpts = + new ConfigCommand.ConfigCommandOptions(toArray(alterOpts, entityOp(brokerId), + asList("--delete-config", String.join(",", configNames)))); + ConfigCommand.alterConfigWithZk(zkClient, deleteOpts, adminZkClient); + verifyConfig(zkClient, brokerId, Collections.emptyMap()); + } + + private Map generateEncodeConfig() { + Map map = new HashMap<>(); + map.put(PASSWORD_ENCODER_SECRET_CONFIG, "encoder-secret"); + map.put(PASSWORD_ENCODER_CIPHER_ALGORITHM_CONFIG, "DES/CBC/PKCS5Padding"); + map.put(PASSWORD_ENCODER_ITERATIONS_CONFIG, "1024"); + map.put(PASSWORD_ENCODER_KEYFACTORY_ALGORITHM_CONFIG, "PBKDF2WithHmacSHA1"); + map.put(PASSWORD_ENCODER_KEY_LENGTH_CONFIG, "64"); + map.put("listener.name.external.ssl.keystore.password", "secret2"); + return map; } private void registerBrokerInZk(KafkaZkClient zkClient, int id) { zkClient.createTopLevelPaths(); SecurityProtocol securityProtocol = SecurityProtocol.PLAINTEXT; - EndPoint endpoint = new EndPoint("localhost", 9092, ListenerName.forSecurityProtocol(securityProtocol), securityProtocol); - BrokerInfo brokerInfo = BrokerInfo.apply(Broker.apply(id, seq(endpoint), scala.None$.empty()), MetadataVersion.latestTesting(), 9192); + EndPoint endpoint = new EndPoint("localhost", 9092, + ListenerName.forSecurityProtocol(securityProtocol), securityProtocol); + BrokerInfo brokerInfo = BrokerInfo.apply(Broker.apply(id, endpoint, + scala.None$.empty()), MetadataVersion.latestTesting(), 9192); zkClient.registerBroker(brokerInfo); } - @SafeVarargs - static Seq seq(T...seq) { - return seq(Arrays.asList(seq)); + private List generateDefaultAlterOpts(String bootstrapServers) { + return asList("--bootstrap-server", bootstrapServers, + "--entity-type", "brokers", + "--entity-name", "0", "--alter"); + } + + private void alterAndVerifyConfig(Admin client, Optional brokerId, Map config) throws Exception { + alterConfigWithKraft(client, brokerId, config); + verifyConfig(client, brokerId, config); + } + + private void alterConfigWithKraft(Admin client, Optional brokerId, Map config) { + String configStr = transferConfigMapToString(config); + ConfigCommand.ConfigCommandOptions addOpts = + new ConfigCommand.ConfigCommandOptions(toArray(alterOpts, entityOp(brokerId), asList("--add-config", configStr))); + ConfigCommand.alterConfig(client, addOpts); + } + + private void verifyConfig(Admin client, Optional brokerId, Map config) throws Exception { + ConfigResource configResource = new ConfigResource(ConfigResource.Type.BROKER, brokerId.orElse(defaultBrokerId)); + TestUtils.waitForCondition(() -> { + Map current = client.describeConfigs(singletonList(configResource)) + .all() + .get() + .values() + .stream() + .flatMap(e -> e.entries().stream()) + .collect(HashMap::new, (map, entry) -> map.put(entry.name(), entry.value()), HashMap::putAll); + return config.entrySet().stream().allMatch(e -> e.getValue().equals(current.get(e.getKey()))); + }, 10000, config + " are not updated"); + } + + private void deleteAndVerifyConfig(Admin client, Optional brokerId, Set config) throws Exception { + ConfigCommand.ConfigCommandOptions deleteOpts = + new ConfigCommand.ConfigCommandOptions(toArray(alterOpts, entityOp(brokerId), + asList("--delete-config", String.join(",", config)))); + ConfigCommand.alterConfig(client, deleteOpts); + verifyConfigDefaultValue(client, brokerId, config); } - @SuppressWarnings({"deprecation"}) - static Seq seq(Collection seq) { - return JavaConverters.asScalaIteratorConverter(seq.iterator()).asScala().toSeq(); + private void verifyConfigDefaultValue(Admin client, Optional brokerId, Set config) throws Exception { + ConfigResource configResource = new ConfigResource(ConfigResource.Type.BROKER, brokerId.orElse(defaultBrokerId)); + TestUtils.waitForCondition(() -> { + Map current = client.describeConfigs(singletonList(configResource)) + .all() + .get() + .values() + .stream() + .flatMap(e -> e.entries().stream()) + .collect(HashMap::new, (map, entry) -> map.put(entry.name(), entry.value()), HashMap::putAll); + return config.stream().allMatch(current::containsKey); + }, 5000, config + " are not updated"); } @SafeVarargs - public static String[] toArray(List... lists) { + private static String[] toArray(List... lists) { return Stream.of(lists).flatMap(List::stream).toArray(String[]::new); } - public static String captureStandardErr(Runnable runnable) { - return captureStandardStream(true, runnable); + private String captureStandardMsg(Runnable runnable) { + return captureStandardStream(runnable); + } + + private String transferConfigMapToString(Map configs) { + return configs.entrySet() + .stream() + .map(e -> e.getKey() + "=" + e.getValue()) + .collect(Collectors.joining(",")); } - private static String captureStandardStream(boolean isErr, Runnable runnable) { + private String captureStandardStream(Runnable runnable) { ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - PrintStream currentStream = isErr ? System.err : System.out; - PrintStream tempStream = new PrintStream(outputStream); - if (isErr) + PrintStream currentStream = System.err; + try (PrintStream tempStream = new PrintStream(outputStream)) { System.setErr(tempStream); - else - System.setOut(tempStream); - try { - runnable.run(); - return outputStream.toString().trim(); - } finally { - if (isErr) + try { + runnable.run(); + return outputStream.toString().trim(); + } finally { System.setErr(currentStream); - else - System.setOut(currentStream); - - tempStream.close(); + } } } } diff --git a/core/src/test/java/kafka/admin/ConfigCommandTest.java b/core/src/test/java/kafka/admin/ConfigCommandTest.java index afb22a06c249b..5968f3706a1c2 100644 --- a/core/src/test/java/kafka/admin/ConfigCommandTest.java +++ b/core/src/test/java/kafka/admin/ConfigCommandTest.java @@ -421,8 +421,8 @@ public void testParseConfigsToBeAddedForAddConfigFile() throws IOException { public void testExpectedEntityTypeNames(List expectedTypes, List expectedNames, List connectOpts, String...args) { ConfigCommand.ConfigCommandOptions createOpts = new ConfigCommand.ConfigCommandOptions(toArray(Arrays.asList(connectOpts.get(0), connectOpts.get(1), "--describe"), Arrays.asList(args))); createOpts.checkArgs(); - assertEquals(createOpts.entityTypes().toSeq(), ConfigCommandIntegrationTest.seq(expectedTypes)); - assertEquals(createOpts.entityNames().toSeq(), ConfigCommandIntegrationTest.seq(expectedNames)); + assertEquals(createOpts.entityTypes().toSeq(), seq(expectedTypes)); + assertEquals(createOpts.entityNames().toSeq(), seq(expectedNames)); } public void doTestOptionEntityTypeNames(boolean zkConfig) { @@ -1710,7 +1710,7 @@ public void testUserClientQuotaOpts() { public void checkEntities(List opts, Map> expectedFetches, List expectedEntityNames) { ConfigCommand.ConfigEntity entity = ConfigCommand.parseEntity(new ConfigCommand.ConfigCommandOptions(toArray(opts, Collections.singletonList("--describe")))); expectedFetches.forEach((name, values) -> - when(zkClient.getAllEntitiesWithConfig(name)).thenReturn(ConfigCommandIntegrationTest.seq(values))); + when(zkClient.getAllEntitiesWithConfig(name)).thenReturn(seq(values))); Seq entities0 = entity.getAllEntities(zkClient); List entities = new ArrayList<>(); entities0.foreach(e -> { @@ -1996,4 +1996,9 @@ public AlterClientQuotasResult alterClientQuotas(Collection Seq seq(Collection seq) { + return JavaConverters.asScalaIteratorConverter(seq.iterator()).asScala().toSeq(); + } } diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 7fda9c6e37cb0..612fd2d416f4e 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -20,6 +20,7 @@ import kafka.cluster.EndPoint; import kafka.cluster.Partition; import kafka.log.UnifiedLog; +import kafka.log.remote.quota.RLMQuotaManagerConfig; import kafka.server.BrokerTopicStats; import kafka.server.KafkaConfig; import kafka.server.StopPartition; @@ -30,6 +31,7 @@ import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.errors.ReplicaNotAvailableException; +import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; @@ -41,6 +43,7 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.common.OffsetAndEpoch; +import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.log.remote.storage.ClassLoaderAwareRemoteStorageManager; import org.apache.kafka.server.log.remote.storage.LogSegmentData; import org.apache.kafka.server.log.remote.storage.NoOpRemoteLogMetadataManager; @@ -126,6 +129,12 @@ import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_CONSUMER_PREFIX; import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_PRODUCER_PREFIX; import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_PARTITIONS_PROP; +import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND; +import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM; +import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS; +import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND; +import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM; +import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS; import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX; import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.DEFAULT_REMOTE_STORAGE_MANAGER_CONFIG_PREFIX; import static org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics.REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC; @@ -180,6 +189,7 @@ public class RemoteLogManagerTest { private RemoteLogManagerConfig remoteLogManagerConfig = null; private BrokerTopicStats brokerTopicStats = null; + private final Metrics metrics = new Metrics(time); private RemoteLogManager remoteLogManager = null; private final TopicIdPartition leaderTopicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("Leader", 0)); @@ -194,7 +204,7 @@ public class RemoteLogManagerTest { private LeaderEpochCheckpointFile checkpoint; private final AtomicLong currentLogStartOffset = new AtomicLong(0L); - private final UnifiedLog mockLog = mock(UnifiedLog.class); + private UnifiedLog mockLog = mock(UnifiedLog.class); private final MockScheduler scheduler = new MockScheduler(time); @@ -207,12 +217,12 @@ void setUp() throws Exception { props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true"); props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP, "100"); remoteLogManagerConfig = createRLMConfig(props); - brokerTopicStats = new BrokerTopicStats(Optional.of(KafkaConfig.fromProps(props))); + brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).remoteLogManagerConfig().enableRemoteStorageSystem()); remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), - brokerTopicStats) { + brokerTopicStats, metrics) { public RemoteStorageManager createRemoteStorageManager() { return remoteStorageManager; } @@ -349,7 +359,7 @@ void testRemoteLogMetadataManagerWithEndpointConfig() { assertEquals(host + ":" + port, capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "bootstrap.servers")); assertEquals(securityProtocol, capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol")); assertEquals(clusterId, capture.getValue().get("cluster.id")); - assertEquals(brokerId, capture.getValue().get(KafkaConfig.BrokerIdProp())); + assertEquals(brokerId, capture.getValue().get(ServerConfigs.BROKER_ID_CONFIG)); } @Test @@ -365,7 +375,8 @@ void testRemoteLogMetadataManagerWithEndpointConfigOverridden() throws IOExcepti time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, - brokerTopicStats) { + brokerTopicStats, + metrics) { public RemoteStorageManager createRemoteStorageManager() { return remoteStorageManager; } @@ -388,7 +399,7 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() { // should be overridden as SSL assertEquals("SSL", capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol")); assertEquals(clusterId, capture.getValue().get("cluster.id")); - assertEquals(brokerId, capture.getValue().get(KafkaConfig.BrokerIdProp())); + assertEquals(brokerId, capture.getValue().get(ServerConfigs.BROKER_ID_CONFIG)); } } @@ -647,12 +658,22 @@ void testCustomMetadataSizeExceedsLimit() throws Exception { assertEquals(1, brokerTopicStats.allTopicsStats().failedRemoteCopyRequestRate().count()); } + @Test + void testLeadershipChangesWithoutRemoteLogManagerConfiguring() { + assertThrows(KafkaException.class, () -> { + remoteLogManager.onLeadershipChange( + Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.singleton(mockPartition(followerTopicIdPartition)), topicIds); + }, "RemoteLogManager is not configured when remote storage system is enabled"); + } + @Test void testRemoteLogManagerTasksAvgIdlePercentAndMetadataCountMetrics() throws Exception { + remoteLogManager.startup(); long oldSegmentStartOffset = 0L; long nextSegmentStartOffset = 150L; int segmentCount = 3; when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); + when(mockLog.parentDir()).thenReturn("dir1"); // leader epoch preparation checkpoint.write(totalEpochEntries); @@ -721,7 +742,7 @@ void testRemoteLogManagerTasksAvgIdlePercentAndMetadataCountMetrics() throws Exc Partition mockLeaderPartition = mockPartition(leaderTopicIdPartition); Partition mockFollowerPartition = mockPartition(followerTopicIdPartition); List list = listRemoteLogSegmentMetadata(leaderTopicIdPartition, segmentCount, 100, 1024, RemoteLogSegmentState.COPY_SEGMENT_FINISHED); - // return 3 metadata and then return 0 to simulate all segments are deleted + // return the metadataList 3 times, then return empty list to simulate all segments are deleted when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition)).thenReturn(list.iterator()).thenReturn(Collections.emptyIterator()); when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 0)).thenReturn(list.iterator()).thenReturn(list.iterator()); when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 1)).thenReturn(list.iterator()); @@ -764,12 +785,128 @@ void testRemoteLogManagerTasksAvgIdlePercentAndMetadataCountMetrics() throws Exc safeLongYammerMetricValue("RemoteLogSizeBytes"))); } + @Test + void testRemoteLogTaskUpdateRemoteLogSegmentMetadataAfterLogDirChanged() throws Exception { + remoteLogManager.startup(); + long oldSegmentStartOffset = 0L; + long nextSegmentStartOffset = 150L; + int segmentCount = 3; + when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); + when(mockLog.parentDir()).thenReturn("dir1"); + + // leader epoch preparation + checkpoint.write(totalEpochEntries); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint); + when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())) + .thenReturn(Optional.of(0L)) + .thenReturn(Optional.of(nextSegmentStartOffset - 1)); + + File tempFile = TestUtils.tempFile(); + File mockProducerSnapshotIndex = TestUtils.tempFile(); + File tempDir = TestUtils.tempDirectory(); + // create 2 log segments, with 0 and 150 as log start offset + LogSegment oldSegment = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(oldSegment.baseOffset()).thenReturn(oldSegmentStartOffset); + when(activeSegment.baseOffset()).thenReturn(nextSegmentStartOffset); + + FileRecords fileRecords = mock(FileRecords.class); + when(oldSegment.log()).thenReturn(fileRecords); + when(fileRecords.file()).thenReturn(tempFile); + when(fileRecords.sizeInBytes()).thenReturn(10); + when(oldSegment.readNextOffset()).thenReturn(nextSegmentStartOffset); + + when(mockLog.activeSegment()).thenReturn(activeSegment); + when(mockLog.logStartOffset()).thenReturn(oldSegmentStartOffset); + when(mockLog.logSegments(anyLong(), anyLong())).thenReturn(JavaConverters.collectionAsScalaIterable(Arrays.asList(oldSegment, activeSegment))); + + ProducerStateManager mockStateManager = mock(ProducerStateManager.class); + when(mockLog.producerStateManager()).thenReturn(mockStateManager); + when(mockStateManager.fetchSnapshot(anyLong())).thenReturn(Optional.of(mockProducerSnapshotIndex)); + when(mockLog.lastStableOffset()).thenReturn(250L); + when(mockLog.logEndOffset()).thenReturn(500L); + Map logProps = new HashMap<>(); + logProps.put("retention.bytes", 100L); + logProps.put("retention.ms", -1L); + LogConfig logConfig = new LogConfig(logProps); + when(mockLog.config()).thenReturn(logConfig); + + OffsetIndex idx = LazyIndex.forOffset(LogFileUtils.offsetIndexFile(tempDir, oldSegmentStartOffset, ""), oldSegmentStartOffset, 1000).get(); + TimeIndex timeIdx = LazyIndex.forTime(LogFileUtils.timeIndexFile(tempDir, oldSegmentStartOffset, ""), oldSegmentStartOffset, 1500).get(); + File txnFile = UnifiedLog.transactionIndexFile(tempDir, oldSegmentStartOffset, ""); + txnFile.createNewFile(); + TransactionIndex txnIndex = new TransactionIndex(oldSegmentStartOffset, txnFile); + when(oldSegment.timeIndex()).thenReturn(timeIdx); + when(oldSegment.offsetIndex()).thenReturn(idx); + when(oldSegment.txnIndex()).thenReturn(txnIndex); + + CompletableFuture dummyFuture = new CompletableFuture<>(); + dummyFuture.complete(null); + when(remoteLogMetadataManager.addRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadata.class))).thenReturn(dummyFuture); + when(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadataUpdate.class))).thenReturn(dummyFuture); + + CountDownLatch copyLogSegmentLatch = new CountDownLatch(1); + doAnswer(ans -> { + // waiting for verification + copyLogSegmentLatch.await(5000, TimeUnit.MILLISECONDS); + return Optional.empty(); + }).when(remoteStorageManager).copyLogSegmentData(any(RemoteLogSegmentMetadata.class), any(LogSegmentData.class)); + + Partition mockLeaderPartition = mockPartition(leaderTopicIdPartition); + List metadataList = listRemoteLogSegmentMetadata(leaderTopicIdPartition, segmentCount, 100, 1024, RemoteLogSegmentState.COPY_SEGMENT_FINISHED); + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition)).thenReturn(metadataList.iterator()); + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 0)).thenReturn(metadataList.iterator()).thenReturn(metadataList.iterator()); + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 1)).thenReturn(metadataList.iterator()); + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 2)).thenReturn(metadataList.iterator()); + + // leadership change to log in dir1 + remoteLogManager.onLeadershipChange(Collections.singleton(mockLeaderPartition), Collections.emptySet(), topicIds); + + TestUtils.waitForCondition(() -> { + ArgumentCaptor argument = ArgumentCaptor.forClass(Long.class); + verify(mockLog, times(1)).updateHighestOffsetInRemoteStorage(argument.capture()); + return 0L == argument.getValue(); + }, "Timed out waiting for updateHighestOffsetInRemoteStorage(0) get invoked for dir1 log"); + + UnifiedLog oldMockLog = mockLog; + Mockito.clearInvocations(oldMockLog); + // simulate altering log dir completes, and the new partition leader changes to the same broker in different log dir (dir2) + mockLog = mock(UnifiedLog.class); + when(mockLog.parentDir()).thenReturn("dir2"); + when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.config()).thenReturn(logConfig); + when(mockLog.logEndOffset()).thenReturn(500L); + + remoteLogManager.onLeadershipChange(Collections.singleton(mockLeaderPartition), Collections.emptySet(), topicIds); + + // after copyLogSegment completes for log (in dir1), updateHighestOffsetInRemoteStorage will be triggered with new offset + // even though the leader replica has changed to log in dir2 + copyLogSegmentLatch.countDown(); + TestUtils.waitForCondition(() -> { + ArgumentCaptor argument = ArgumentCaptor.forClass(Long.class); + verify(oldMockLog, times(1)).updateHighestOffsetInRemoteStorage(argument.capture()); + return nextSegmentStartOffset - 1 == argument.getValue(); + }, "Timed out waiting for updateHighestOffsetInRemoteStorage(149) get invoked for dir1 log"); + + // On the next run of RLMTask, the log in dir2 will be picked and start by updateHighestOffsetInRemoteStorage to the expected offset + TestUtils.waitForCondition(() -> { + ArgumentCaptor argument = ArgumentCaptor.forClass(Long.class); + verify(mockLog, times(1)).updateHighestOffsetInRemoteStorage(argument.capture()); + return nextSegmentStartOffset - 1 == argument.getValue(); + }, "Timed out waiting for updateHighestOffsetInRemoteStorage(149) get invoked for dir2 log"); + + } + @Test void testRemoteLogManagerRemoteMetrics() throws Exception { + remoteLogManager.startup(); long oldestSegmentStartOffset = 0L; long olderSegmentStartOffset = 75L; long nextSegmentStartOffset = 150L; when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); + when(mockLog.parentDir()).thenReturn("dir1"); // leader epoch preparation checkpoint.write(totalEpochEntries); @@ -1124,7 +1261,7 @@ void testGetClassLoaderAwareRemoteStorageManager() throws Exception { new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time, t -> Optional.empty(), (topicPartition, offset) -> { }, - brokerTopicStats) { + brokerTopicStats, metrics) { public RemoteStorageManager createRemoteStorageManager() { return rsmManager; } @@ -1149,6 +1286,7 @@ private void verifyNotInCache(TopicIdPartition... topicIdPartitions) { @Test void testTopicIdCacheUpdates() throws RemoteStorageException { + remoteLogManager.startup(); Partition mockLeaderPartition = mockPartition(leaderTopicIdPartition); Partition mockFollowerPartition = mockPartition(followerTopicIdPartition); @@ -1173,6 +1311,7 @@ void testTopicIdCacheUpdates() throws RemoteStorageException { @Test void testFetchRemoteLogSegmentMetadata() throws RemoteStorageException { + remoteLogManager.startup(); remoteLogManager.onLeadershipChange( Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.singleton(mockPartition(followerTopicIdPartition)), topicIds); remoteLogManager.fetchRemoteLogSegmentMetadata(leaderTopicIdPartition.topicPartition(), 10, 100L); @@ -1186,6 +1325,7 @@ void testFetchRemoteLogSegmentMetadata() throws RemoteStorageException { @Test void testOnLeadershipChangeWillInvokeHandleLeaderOrFollowerPartitions() { + remoteLogManager.startup(); RemoteLogManager spyRemoteLogManager = spy(remoteLogManager); spyRemoteLogManager.onLeadershipChange( Collections.emptySet(), Collections.singleton(mockPartition(followerTopicIdPartition)), topicIds); @@ -1220,6 +1360,7 @@ void testRLMTaskShouldSetLeaderEpochCorrectly() { @Test void testFindOffsetByTimestamp() throws IOException, RemoteStorageException { + remoteLogManager.startup(); TopicPartition tp = leaderTopicIdPartition.topicPartition(); long ts = time.milliseconds(); @@ -1253,6 +1394,7 @@ void testFindOffsetByTimestamp() throws IOException, RemoteStorageException { @Test void testFindOffsetByTimestampWithInvalidEpochSegments() throws IOException, RemoteStorageException { + remoteLogManager.startup(); TopicPartition tp = leaderTopicIdPartition.topicPartition(); long ts = time.milliseconds(); @@ -1284,6 +1426,7 @@ void testFindOffsetByTimestampWithInvalidEpochSegments() throws IOException, Rem @Test void testFindOffsetByTimestampWithSegmentNotReady() throws IOException, RemoteStorageException { + remoteLogManager.startup(); TopicPartition tp = leaderTopicIdPartition.topicPartition(); long ts = time.milliseconds(); @@ -1378,7 +1521,7 @@ public void testRemoveMetricsOnClose() throws IOException { MockedConstruction mockMetricsGroupCtor = mockConstruction(KafkaMetricsGroup.class); try { RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, - time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats) { + time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, metrics) { public RemoteStorageManager createRemoteStorageManager() { return remoteStorageManager; } @@ -1689,6 +1832,7 @@ public void testRemoteSizeTime() { @Test public void testStopPartitionsWithoutDeletion() throws RemoteStorageException { + remoteLogManager.startup(); BiConsumer errorHandler = (topicPartition, throwable) -> fail("shouldn't be called"); Set partitions = new HashSet<>(); partitions.add(new StopPartition(leaderTopicIdPartition.topicPartition(), true, false)); @@ -1708,6 +1852,7 @@ public void testStopPartitionsWithoutDeletion() throws RemoteStorageException { @Test public void testStopPartitionsWithDeletion() throws RemoteStorageException { + remoteLogManager.startup(); BiConsumer errorHandler = (topicPartition, ex) -> fail("shouldn't be called: " + ex); Set partitions = new HashSet<>(); @@ -1770,7 +1915,7 @@ public void testFindLogStartOffset() throws RemoteStorageException, IOException try (RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, - brokerTopicStats) { + brokerTopicStats, metrics) { public RemoteLogMetadataManager createRemoteLogMetadataManager() { return remoteLogMetadataManager; } @@ -1795,7 +1940,7 @@ public void testFindLogStartOffsetFallbackToLocalLogStartOffsetWhenRemoteIsEmpty try (RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, - brokerTopicStats) { + brokerTopicStats, metrics) { public RemoteLogMetadataManager createRemoteLogMetadataManager() { return remoteLogMetadataManager; } @@ -1829,7 +1974,7 @@ public void testLogStartOffsetUpdatedOnStartup() throws RemoteStorageException, try (RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> logStartOffset.set(offset), - brokerTopicStats) { + brokerTopicStats, metrics) { public RemoteLogMetadataManager createRemoteLogMetadataManager() { return remoteLogMetadataManager; } @@ -2192,7 +2337,7 @@ public void testDeleteRetentionMsOnExpiredSegment() throws RemoteStorageExceptio try (RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> logStartOffset.set(offset), - brokerTopicStats) { + brokerTopicStats, metrics) { public RemoteStorageManager createRemoteStorageManager() { return remoteStorageManager; } @@ -2338,7 +2483,8 @@ public void testReadForMissingFirstBatchInRemote() throws RemoteStorageException time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, - brokerTopicStats) { + brokerTopicStats, + metrics) { public RemoteStorageManager createRemoteStorageManager() { return rsmManager; } @@ -2410,7 +2556,8 @@ public void testReadForFirstBatchMoreThanMaxFetchBytes(boolean minOneMessage) th time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, - brokerTopicStats) { + brokerTopicStats, + metrics) { public RemoteStorageManager createRemoteStorageManager() { return rsmManager; } @@ -2495,7 +2642,8 @@ public void testReadForFirstBatchInLogCompaction() throws RemoteStorageException tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, - brokerTopicStats) { + brokerTopicStats, + metrics) { public RemoteStorageManager createRemoteStorageManager() { return rsmManager; } @@ -2527,6 +2675,46 @@ int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, l } } + + @Test + public void testCopyQuotaManagerConfig() { + Properties defaultProps = new Properties(); + RemoteLogManagerConfig defaultRlmConfig = createRLMConfig(defaultProps); + RLMQuotaManagerConfig defaultConfig = RemoteLogManager.copyQuotaManagerConfig(defaultRlmConfig); + assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, defaultConfig.quotaBytesPerSecond()); + assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM, defaultConfig.numQuotaSamples()); + assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS, defaultConfig.quotaWindowSizeSeconds()); + + Properties customProps = new Properties(); + customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, 100); + customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM_PROP, 31); + customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS_PROP, 1); + RemoteLogManagerConfig rlmConfig = createRLMConfig(customProps); + RLMQuotaManagerConfig rlmCopyQuotaManagerConfig = RemoteLogManager.copyQuotaManagerConfig(rlmConfig); + assertEquals(100L, rlmCopyQuotaManagerConfig.quotaBytesPerSecond()); + assertEquals(31, rlmCopyQuotaManagerConfig.numQuotaSamples()); + assertEquals(1, rlmCopyQuotaManagerConfig.quotaWindowSizeSeconds()); + } + + @Test + public void testFetchQuotaManagerConfig() { + Properties defaultProps = new Properties(); + RemoteLogManagerConfig defaultRlmConfig = createRLMConfig(defaultProps); + RLMQuotaManagerConfig defaultConfig = RemoteLogManager.fetchQuotaManagerConfig(defaultRlmConfig); + assertEquals(DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND, defaultConfig.quotaBytesPerSecond()); + assertEquals(DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM, defaultConfig.numQuotaSamples()); + assertEquals(DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS, defaultConfig.quotaWindowSizeSeconds()); + + Properties customProps = new Properties(); + customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP, 100); + customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM_PROP, 31); + customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS_PROP, 1); + RemoteLogManagerConfig rlmConfig = createRLMConfig(customProps); + RLMQuotaManagerConfig rlmFetchQuotaManagerConfig = RemoteLogManager.fetchQuotaManagerConfig(rlmConfig); + assertEquals(100L, rlmFetchQuotaManagerConfig.quotaBytesPerSecond()); + assertEquals(31, rlmFetchQuotaManagerConfig.numQuotaSamples()); + assertEquals(1, rlmFetchQuotaManagerConfig.quotaWindowSizeSeconds()); + } @Test public void testEpochEntriesAsByteBuffer() throws Exception { diff --git a/core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java b/core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java index d1d35b5e5df17..bff58364b9d1f 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogReaderTest.java @@ -17,11 +17,9 @@ package kafka.log.remote; import kafka.server.BrokerTopicStats; -import kafka.server.KafkaConfig; import kafka.utils.TestUtils; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.record.Records; -import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig; import org.apache.kafka.server.log.remote.storage.RemoteStorageException; import org.apache.kafka.storage.internals.log.FetchDataInfo; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; @@ -32,8 +30,6 @@ import org.mockito.ArgumentCaptor; import java.io.IOException; -import java.util.Optional; -import java.util.Properties; import java.util.function.Consumer; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -55,9 +51,7 @@ public class RemoteLogReaderTest { @BeforeEach public void setUp() { TestUtils.clearYammerMetrics(); - Properties props = kafka.utils.TestUtils.createDummyBrokerConfig(); - props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true"); - brokerTopicStats = new BrokerTopicStats(Optional.of(KafkaConfig.fromProps(props))); + brokerTopicStats = new BrokerTopicStats(true); } @Test diff --git a/core/src/test/java/kafka/log/remote/quota/RLMQuotaManagerTest.java b/core/src/test/java/kafka/log/remote/quota/RLMQuotaManagerTest.java new file mode 100644 index 0000000000000..2dd5ddaaf2a01 --- /dev/null +++ b/core/src/test/java/kafka/log/remote/quota/RLMQuotaManagerTest.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.log.remote.quota; + +import kafka.server.QuotaType; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Quota; +import org.apache.kafka.common.utils.MockTime; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class RLMQuotaManagerTest { + private final MockTime time = new MockTime(); + private final Metrics metrics = new Metrics(new MetricConfig(), Collections.emptyList(), time); + private static final QuotaType QUOTA_TYPE = QuotaType.RLMFetch$.MODULE$; + private static final String DESCRIPTION = "Tracking byte rate"; + + @Test + public void testQuotaExceeded() { + RLMQuotaManager quotaManager = new RLMQuotaManager( + new RLMQuotaManagerConfig(50, 11, 1), metrics, QUOTA_TYPE, DESCRIPTION, time); + + assertFalse(quotaManager.isQuotaExceeded()); + quotaManager.record(500); + // Move clock by 1 sec, quota is violated + moveClock(1); + assertTrue(quotaManager.isQuotaExceeded()); + + // Move clock by another 8 secs, quota is still violated for the window + moveClock(8); + assertTrue(quotaManager.isQuotaExceeded()); + + // Move clock by 1 sec, quota is no more violated + moveClock(1); + assertFalse(quotaManager.isQuotaExceeded()); + } + + @Test + public void testQuotaUpdate() { + RLMQuotaManager quotaManager = new RLMQuotaManager( + new RLMQuotaManagerConfig(50, 11, 1), metrics, QUOTA_TYPE, DESCRIPTION, time); + + assertFalse(quotaManager.isQuotaExceeded()); + quotaManager.record(51); + assertTrue(quotaManager.isQuotaExceeded()); + + Map fetchQuotaMetrics = metrics.metrics().entrySet().stream() + .filter(entry -> entry.getKey().name().equals("byte-rate") && entry.getKey().group().equals(QUOTA_TYPE.toString())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + Map nonQuotaMetrics = metrics.metrics().entrySet().stream() + .filter(entry -> !entry.getKey().name().equals("byte-rate") || !entry.getKey().group().equals(QUOTA_TYPE.toString())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + assertEquals(1, fetchQuotaMetrics.size()); + assertFalse(nonQuotaMetrics.isEmpty()); + + Map configForQuotaMetricsBeforeUpdate = extractMetricConfig(fetchQuotaMetrics); + Map configForNonQuotaMetricsBeforeUpdate = extractMetricConfig(nonQuotaMetrics); + + // Update quota to 60, quota is no more violated + Quota quota60Bytes = new Quota(60, true); + quotaManager.updateQuota(quota60Bytes); + assertFalse(quotaManager.isQuotaExceeded()); + + // Verify quota metrics were updated + Map configForQuotaMetricsAfterFirstUpdate = extractMetricConfig(fetchQuotaMetrics); + assertNotEquals(configForQuotaMetricsBeforeUpdate, configForQuotaMetricsAfterFirstUpdate); + fetchQuotaMetrics.values().forEach(metric -> assertEquals(quota60Bytes, metric.config().quota())); + // Verify non quota metrics are unchanged + assertEquals(configForNonQuotaMetricsBeforeUpdate, extractMetricConfig(nonQuotaMetrics)); + + // Update quota to 40, quota is violated again + Quota quota40Bytes = new Quota(40, true); + quotaManager.updateQuota(quota40Bytes); + assertTrue(quotaManager.isQuotaExceeded()); + + // Verify quota metrics were updated + assertNotEquals(configForQuotaMetricsAfterFirstUpdate, extractMetricConfig(fetchQuotaMetrics)); + fetchQuotaMetrics.values().forEach(metric -> assertEquals(quota40Bytes, metric.config().quota())); + // Verify non quota metrics are unchanged + assertEquals(configForNonQuotaMetricsBeforeUpdate, extractMetricConfig(nonQuotaMetrics)); + } + + private void moveClock(int secs) { + time.setCurrentTimeMs(time.milliseconds() + secs * 1000L); + } + + private Map extractMetricConfig(Map metrics) { + return metrics.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().config())); + } +} \ No newline at end of file diff --git a/core/src/test/java/kafka/test/ClusterConfig.java b/core/src/test/java/kafka/test/ClusterConfig.java index c116a7d5050df..b72bb1a5787d7 100644 --- a/core/src/test/java/kafka/test/ClusterConfig.java +++ b/core/src/test/java/kafka/test/ClusterConfig.java @@ -19,6 +19,7 @@ import kafka.test.annotation.Type; import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.server.common.Features; import org.apache.kafka.server.common.MetadataVersion; import java.io.File; @@ -58,13 +59,15 @@ public class ClusterConfig { private final Map saslClientProperties; private final List tags; private final Map> perServerProperties; + private final Map features; @SuppressWarnings("checkstyle:ParameterNumber") private ClusterConfig(Set types, int brokers, int controllers, int disksPerBroker, boolean autoStart, SecurityProtocol securityProtocol, String listenerName, File trustStoreFile, MetadataVersion metadataVersion, Map serverProperties, Map producerProperties, Map consumerProperties, Map adminClientProperties, Map saslServerProperties, - Map saslClientProperties, Map> perServerProperties, List tags) { + Map saslClientProperties, Map> perServerProperties, List tags, + Map features) { // do fail fast. the following values are invalid for both zk and kraft modes. if (brokers < 0) throw new IllegalArgumentException("Number of brokers must be greater or equal to zero."); if (controllers < 0) throw new IllegalArgumentException("Number of controller must be greater or equal to zero."); @@ -87,6 +90,7 @@ private ClusterConfig(Set types, int brokers, int controllers, int disksPe this.saslClientProperties = Objects.requireNonNull(saslClientProperties); this.perServerProperties = Objects.requireNonNull(perServerProperties); this.tags = Objects.requireNonNull(tags); + this.features = Objects.requireNonNull(features); } public Set clusterTypes() { @@ -157,6 +161,10 @@ public List tags() { return tags; } + public Map features() { + return features; + } + public Set displayTags() { Set displayTags = new LinkedHashSet<>(tags); displayTags.add("MetadataVersion=" + metadataVersion); @@ -198,7 +206,8 @@ public static Builder builder(ClusterConfig clusterConfig) { .setSaslServerProperties(clusterConfig.saslServerProperties) .setSaslClientProperties(clusterConfig.saslClientProperties) .setPerServerProperties(clusterConfig.perServerProperties) - .setTags(clusterConfig.tags); + .setTags(clusterConfig.tags) + .setFeatures(clusterConfig.features); } public static class Builder { @@ -219,6 +228,7 @@ public static class Builder { private Map saslClientProperties = Collections.emptyMap(); private Map> perServerProperties = Collections.emptyMap(); private List tags = Collections.emptyList(); + private Map features = Collections.emptyMap(); private Builder() {} @@ -309,11 +319,16 @@ public Builder setTags(List tags) { return this; } + public Builder setFeatures(Map features) { + this.features = Collections.unmodifiableMap(features); + return this; + } + public ClusterConfig build() { return new ClusterConfig(types, brokers, controllers, disksPerBroker, autoStart, securityProtocol, listenerName, trustStoreFile, metadataVersion, serverProperties, producerProperties, consumerProperties, adminClientProperties, saslServerProperties, saslClientProperties, - perServerProperties, tags); + perServerProperties, tags, features); } } } diff --git a/core/src/test/java/kafka/test/ClusterInstance.java b/core/src/test/java/kafka/test/ClusterInstance.java index 8cb5ae3d2152b..03e19f39b62b8 100644 --- a/core/src/test/java/kafka/test/ClusterInstance.java +++ b/core/src/test/java/kafka/test/ClusterInstance.java @@ -40,7 +40,6 @@ import static org.apache.kafka.clients.consumer.GroupProtocol.CLASSIC; import static org.apache.kafka.clients.consumer.GroupProtocol.CONSUMER; import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG; -import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG; public interface ClusterInstance { @@ -159,9 +158,7 @@ default Set supportedGroupProtocols() { Set supportedGroupProtocols = new HashSet<>(); supportedGroupProtocols.add(CLASSIC); - // KafkaConfig#isNewGroupCoordinatorEnabled check both NEW_GROUP_COORDINATOR_ENABLE_CONFIG and GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG - if (serverProperties.getOrDefault(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "").equals("true") || - serverProperties.getOrDefault(GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "").contains("consumer")) { + if (serverProperties.getOrDefault(GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "").contains("consumer")) { supportedGroupProtocols.add(CONSUMER); } diff --git a/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java b/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java index db2215a9dd379..c8b53f8b8a2ba 100644 --- a/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java +++ b/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java @@ -185,25 +185,18 @@ public void testNoAutoStart() { @ClusterTest public void testDefaults(ClusterInstance clusterInstance) { - Assertions.assertEquals(MetadataVersion.IBP_3_8_IV0, clusterInstance.config().metadataVersion()); + Assertions.assertEquals(MetadataVersion.IBP_4_0_IVO, clusterInstance.config().metadataVersion()); } @ClusterTests({ - @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { - @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "true"), - }), - @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { + @ClusterTest(types = {Type.KRAFT, Type.CO_KRAFT}, serverProperties = { @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer"), }), - @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { + @ClusterTest(types = {Type.KRAFT, Type.CO_KRAFT}, serverProperties = { @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "true"), @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer"), }), - @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { - @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "true"), - @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic"), - }), - @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { + @ClusterTest(types = {Type.KRAFT, Type.CO_KRAFT}, serverProperties = { @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "false"), @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer"), }, tags = {"disable-new-coordinator-and-enable-new-consumer-rebalance-coordinator"}), @@ -217,12 +210,19 @@ public void testSupportedNewGroupProtocols(ClusterInstance clusterInstance) { } @ClusterTests({ + @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { + @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "true"), + }), @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "false"), }), @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic"), }), + @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { + @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "true"), + @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic"), + }), @ClusterTest(types = {Type.ZK, Type.KRAFT, Type.CO_KRAFT}, serverProperties = { @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, value = "false"), @ClusterConfigProperty(key = GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic"), diff --git a/core/src/test/java/kafka/test/annotation/ClusterFeature.java b/core/src/test/java/kafka/test/annotation/ClusterFeature.java new file mode 100644 index 0000000000000..ab72f13288169 --- /dev/null +++ b/core/src/test/java/kafka/test/annotation/ClusterFeature.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.test.annotation; + +import org.apache.kafka.server.common.Features; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Documented +@Target({ElementType.ANNOTATION_TYPE}) +@Retention(RetentionPolicy.RUNTIME) +public @interface ClusterFeature { + Features feature(); + short version(); +} diff --git a/core/src/test/java/kafka/test/annotation/ClusterTest.java b/core/src/test/java/kafka/test/annotation/ClusterTest.java index 9364ce690ea5c..bd95249b4f457 100644 --- a/core/src/test/java/kafka/test/annotation/ClusterTest.java +++ b/core/src/test/java/kafka/test/annotation/ClusterTest.java @@ -40,8 +40,9 @@ AutoStart autoStart() default AutoStart.DEFAULT; SecurityProtocol securityProtocol() default SecurityProtocol.PLAINTEXT; String listener() default ""; - MetadataVersion metadataVersion() default MetadataVersion.IBP_3_8_IV0; + MetadataVersion metadataVersion() default MetadataVersion.IBP_4_0_IVO; ClusterConfigProperty[] serverProperties() default {}; // users can add tags that they want to display in test String[] tags() default {}; + ClusterFeature[] features() default {}; } diff --git a/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java index 2290a0a99eb18..3e6b5a7d66095 100644 --- a/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java +++ b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java @@ -18,6 +18,7 @@ package kafka.test.junit; import kafka.test.ClusterConfig; +import kafka.test.annotation.ClusterFeature; import kafka.test.annotation.ClusterTest; import kafka.test.annotation.ClusterTestDefaults; import kafka.test.annotation.ClusterTests; @@ -25,6 +26,7 @@ import kafka.test.annotation.ClusterConfigProperty; import kafka.test.annotation.Type; import kafka.test.annotation.AutoStart; +import org.apache.kafka.server.common.Features; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.api.extension.TestTemplateInvocationContext; import org.junit.jupiter.api.extension.TestTemplateInvocationContextProvider; @@ -172,6 +174,10 @@ private List processClusterTestInternal(Extension .filter(e -> e.id() != -1) .collect(Collectors.groupingBy(ClusterConfigProperty::id, Collectors.mapping(Function.identity(), Collectors.toMap(ClusterConfigProperty::key, ClusterConfigProperty::value, (a, b) -> b)))); + + Map features = Arrays.stream(annot.features()) + .collect(Collectors.toMap(ClusterFeature::feature, ClusterFeature::version)); + ClusterConfig config = ClusterConfig.builder() .setTypes(new HashSet<>(Arrays.asList(types))) .setBrokers(annot.brokers() == 0 ? defaults.brokers() : annot.brokers()) @@ -184,6 +190,7 @@ private List processClusterTestInternal(Extension .setSecurityProtocol(annot.securityProtocol()) .setMetadataVersion(annot.metadataVersion()) .setTags(Arrays.asList(annot.tags())) + .setFeatures(features) .build(); return Arrays.stream(types).map(type -> type.invocationContexts(context.getRequiredTestMethod().getName(), config)) diff --git a/core/src/test/java/kafka/test/junit/ClusterTestExtensionsUnitTest.java b/core/src/test/java/kafka/test/junit/ClusterTestExtensionsUnitTest.java index 7a1ae920a6f44..c0944080547d6 100644 --- a/core/src/test/java/kafka/test/junit/ClusterTestExtensionsUnitTest.java +++ b/core/src/test/java/kafka/test/junit/ClusterTestExtensionsUnitTest.java @@ -17,29 +17,64 @@ package kafka.test.junit; +import kafka.test.ClusterConfig; import kafka.test.annotation.ClusterTemplate; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.extension.ExtensionContext; + +import java.lang.reflect.Method; +import java.util.Collections; +import java.util.List; + import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; public class ClusterTestExtensionsUnitTest { + + static List cfgEmpty() { + return Collections.emptyList(); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private ExtensionContext buildExtensionContext(String methodName) throws Exception { + ExtensionContext extensionContext = mock(ExtensionContext.class); + Class clazz = ClusterTestExtensionsUnitTest.class; + Method method = clazz.getDeclaredMethod(methodName); + when(extensionContext.getRequiredTestClass()).thenReturn(clazz); + when(extensionContext.getRequiredTestMethod()).thenReturn(method); + return extensionContext; + } + @Test - void testProcessClusterTemplate() { + void testProcessClusterTemplate() throws Exception { ClusterTestExtensions ext = new ClusterTestExtensions(); - ExtensionContext context = mock(ExtensionContext.class); + ExtensionContext context = buildExtensionContext("cfgEmpty"); ClusterTemplate annot = mock(ClusterTemplate.class); - when(annot.value()).thenReturn("").thenReturn(" "); + when(annot.value()).thenReturn("").thenReturn(" ").thenReturn("cfgEmpty"); + + Assertions.assertEquals( + "ClusterTemplate value can't be empty string.", + Assertions.assertThrows(IllegalStateException.class, () -> + ext.processClusterTemplate(context, annot) + ).getMessage() + ); + - Assertions.assertThrows(IllegalStateException.class, () -> - ext.processClusterTemplate(context, annot) + Assertions.assertEquals( + "ClusterTemplate value can't be empty string.", + Assertions.assertThrows(IllegalStateException.class, () -> + ext.processClusterTemplate(context, annot) + ).getMessage() ); - Assertions.assertThrows(IllegalStateException.class, () -> - ext.processClusterTemplate(context, annot) + Assertions.assertEquals( + "ClusterConfig generator method should provide at least one config", + Assertions.assertThrows(IllegalStateException.class, () -> + ext.processClusterTemplate(context, annot) + ).getMessage() ); } } diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java index df8990b0c92f2..3a7bac5aad5ea 100644 --- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java @@ -28,15 +28,20 @@ import kafka.testkit.TestKitNodes; import kafka.zk.EmbeddedZookeeper; import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.metadata.BrokerState; +import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.extension.AfterTestExecutionCallback; import org.junit.jupiter.api.extension.BeforeTestExecutionCallback; import org.junit.jupiter.api.extension.Extension; import org.junit.jupiter.api.extension.TestTemplateInvocationContext; import scala.compat.java8.OptionConverters; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -48,7 +53,6 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -67,15 +71,11 @@ public class RaftClusterInvocationContext implements TestTemplateInvocationConte private final String baseDisplayName; private final ClusterConfig clusterConfig; - private final AtomicReference clusterReference; - private final AtomicReference zkReference; private final boolean isCombined; public RaftClusterInvocationContext(String baseDisplayName, ClusterConfig clusterConfig, boolean isCombined) { this.baseDisplayName = baseDisplayName; this.clusterConfig = clusterConfig; - this.clusterReference = new AtomicReference<>(); - this.zkReference = new AtomicReference<>(); this.isCombined = isCombined; } @@ -86,67 +86,43 @@ public String getDisplayName(int invocationIndex) { @Override public List getAdditionalExtensions() { - RaftClusterInstance clusterInstance = new RaftClusterInstance(clusterReference, zkReference, clusterConfig, isCombined); + RaftClusterInstance clusterInstance = new RaftClusterInstance(clusterConfig, isCombined); return Arrays.asList( - (BeforeTestExecutionCallback) context -> { - TestKitNodes nodes = new TestKitNodes.Builder(). - setBootstrapMetadataVersion(clusterConfig.metadataVersion()). - setCombined(isCombined). - setNumBrokerNodes(clusterConfig.numBrokers()). - setPerServerProperties(clusterConfig.perServerOverrideProperties()). - setNumDisksPerBroker(clusterConfig.numDisksPerBroker()). - setNumControllerNodes(clusterConfig.numControllers()).build(); - KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder(nodes); - - if (Boolean.parseBoolean(clusterConfig.serverProperties().getOrDefault("zookeeper.metadata.migration.enable", "false"))) { - zkReference.set(new EmbeddedZookeeper()); - builder.setConfigProp("zookeeper.connect", String.format("localhost:%d", zkReference.get().port())); - } - // Copy properties into the TestKit builder - clusterConfig.serverProperties().forEach(builder::setConfigProp); - // KAFKA-12512 need to pass security protocol and listener name here - KafkaClusterTestKit cluster = builder.build(); - clusterReference.set(cluster); - cluster.format(); - if (clusterConfig.isAutoStart()) { - cluster.startup(); - kafka.utils.TestUtils.waitUntilTrue( - () -> cluster.brokers().get(0).brokerState() == BrokerState.RUNNING, - () -> "Broker never made it to RUNNING state.", - org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, - 100L); - } - }, - (AfterTestExecutionCallback) context -> clusterInstance.stop(), - new ClusterInstanceParameterResolver(clusterInstance) + (BeforeTestExecutionCallback) context -> { + clusterInstance.format(); + if (clusterConfig.isAutoStart()) { + clusterInstance.start(); + } + }, + (AfterTestExecutionCallback) context -> clusterInstance.stop(), + new ClusterInstanceParameterResolver(clusterInstance) ); } public static class RaftClusterInstance implements ClusterInstance { - private final AtomicReference clusterReference; - private final AtomicReference zkReference; private final ClusterConfig clusterConfig; final AtomicBoolean started = new AtomicBoolean(false); final AtomicBoolean stopped = new AtomicBoolean(false); + final AtomicBoolean formated = new AtomicBoolean(false); private final ConcurrentLinkedQueue admins = new ConcurrentLinkedQueue<>(); + private EmbeddedZookeeper embeddedZookeeper; + private KafkaClusterTestKit clusterTestKit; private final boolean isCombined; - RaftClusterInstance(AtomicReference clusterReference, AtomicReference zkReference, ClusterConfig clusterConfig, boolean isCombined) { - this.clusterReference = clusterReference; - this.zkReference = zkReference; + RaftClusterInstance(ClusterConfig clusterConfig, boolean isCombined) { this.clusterConfig = clusterConfig; this.isCombined = isCombined; } @Override public String bootstrapServers() { - return clusterReference.get().bootstrapServers(); + return clusterTestKit.bootstrapServers(); } @Override public String bootstrapControllers() { - return clusterReference.get().bootstrapControllers(); + return clusterTestKit.bootstrapControllers(); } @Override @@ -193,25 +169,30 @@ public Set controllerIds() { @Override public KafkaClusterTestKit getUnderlying() { - return clusterReference.get(); + return clusterTestKit; } @Override public Admin createAdminClient(Properties configOverrides) { - Admin admin = Admin.create(clusterReference.get(). - newClientPropertiesBuilder(configOverrides).build()); + Admin admin = Admin.create(clusterTestKit.newClientPropertiesBuilder(configOverrides).build()); admins.add(admin); return admin; } @Override public void start() { - if (started.compareAndSet(false, true)) { - try { - clusterReference.get().startup(); - } catch (Exception e) { - throw new RuntimeException("Failed to start Raft server", e); + try { + format(); + if (started.compareAndSet(false, true)) { + clusterTestKit.startup(); + kafka.utils.TestUtils.waitUntilTrue( + () -> this.clusterTestKit.brokers().get(0).brokerState() == BrokerState.RUNNING, + () -> "Broker never made it to RUNNING state.", + org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, + 100L); } + } catch (Exception e) { + throw new RuntimeException("Failed to start Raft server", e); } } @@ -220,9 +201,9 @@ public void stop() { if (stopped.compareAndSet(false, true)) { admins.forEach(admin -> Utils.closeQuietly(admin, "admin")); admins.clear(); - Utils.closeQuietly(clusterReference.get(), "cluster"); - if (zkReference.get() != null) { - Utils.closeQuietly(zkReference.get(), "zk"); + Utils.closeQuietly(clusterTestKit, "cluster"); + if (embeddedZookeeper != null) { + Utils.closeQuietly(embeddedZookeeper, "zk"); } } } @@ -240,27 +221,64 @@ public void startBroker(int brokerId) { @Override public void waitForReadyBrokers() throws InterruptedException { try { - clusterReference.get().waitForReadyBrokers(); + clusterTestKit.waitForReadyBrokers(); } catch (ExecutionException e) { throw new AssertionError("Failed while waiting for brokers to become ready", e); } } - private BrokerServer findBrokerOrThrow(int brokerId) { - return Optional.ofNullable(clusterReference.get().brokers().get(brokerId)) - .orElseThrow(() -> new IllegalArgumentException("Unknown brokerId " + brokerId)); - } @Override public Map brokers() { - return clusterReference.get().brokers().entrySet() + return clusterTestKit.brokers().entrySet() .stream() .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } @Override public Map controllers() { - return Collections.unmodifiableMap(clusterReference.get().controllers()); + return Collections.unmodifiableMap(clusterTestKit.controllers()); + } + + public void format() throws Exception { + if (formated.compareAndSet(false, true)) { + List records = new ArrayList<>(); + records.add( + new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(clusterConfig.metadataVersion().featureLevel()), (short) 0)); + + clusterConfig.features().forEach((feature, version) -> { + records.add( + new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(feature.featureName()). + setFeatureLevel(version), (short) 0)); + }); + + TestKitNodes nodes = new TestKitNodes.Builder() + .setBootstrapMetadata(BootstrapMetadata.fromRecords(records, "testkit")) + .setCombined(isCombined) + .setNumBrokerNodes(clusterConfig.numBrokers()) + .setNumDisksPerBroker(clusterConfig.numDisksPerBroker()) + .setPerServerProperties(clusterConfig.perServerOverrideProperties()) + .setNumControllerNodes(clusterConfig.numControllers()).build(); + KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder(nodes); + if (Boolean.parseBoolean(clusterConfig.serverProperties() + .getOrDefault("zookeeper.metadata.migration.enable", "false"))) { + this.embeddedZookeeper = new EmbeddedZookeeper(); + builder.setConfigProp("zookeeper.connect", String.format("localhost:%d", embeddedZookeeper.port())); + } + // Copy properties into the TestKit builder + clusterConfig.serverProperties().forEach(builder::setConfigProp); + // KAFKA-12512 need to pass security protocol and listener name here + this.clusterTestKit = builder.build(); + this.clusterTestKit.format(); + } + } + + private BrokerServer findBrokerOrThrow(int brokerId) { + return Optional.ofNullable(clusterTestKit.brokers().get(brokerId)) + .orElseThrow(() -> new IllegalArgumentException("Unknown brokerId " + brokerId)); } } diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index 4d34ce040142b..5365652a5fcb5 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -23,7 +23,6 @@ import kafka.server.FaultHandlerFactory; import kafka.server.SharedServer; import kafka.server.KafkaConfig; -import kafka.server.KafkaConfig$; import kafka.server.KafkaRaftServer; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClientConfig; @@ -41,6 +40,7 @@ import org.apache.kafka.raft.QuorumConfig; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.config.KRaftConfigs; +import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.fault.FaultHandler; import org.apache.kafka.server.fault.MockFaultHandler; import org.apache.kafka.storage.internals.log.CleanerConfig; @@ -213,8 +213,8 @@ private KafkaConfig createNodeConfig(TestKitNode node) { if (controllerNode != null) { props.putAll(controllerNode.propertyOverrides()); } - props.putIfAbsent(KafkaConfig$.MODULE$.UnstableMetadataVersionsEnableProp(), "true"); - props.putIfAbsent(KafkaConfig$.MODULE$.UnstableApiVersionsEnableProp(), "true"); + props.putIfAbsent(ServerConfigs.UNSTABLE_FEATURE_VERSIONS_ENABLE_CONFIG, "true"); + props.putIfAbsent(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true"); return new KafkaConfig(props, false, Option.empty()); } diff --git a/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala b/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala index 79729bae5147a..90f719dff8c8c 100644 --- a/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala +++ b/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala @@ -25,6 +25,7 @@ import org.apache.kafka.common.message.ApiMessageType import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.ApiVersionsResponse import org.apache.kafka.network.SocketServerConfigs +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotNull, assertTrue} import org.junit.jupiter.api.Timeout import org.junit.jupiter.params.ParameterizedTest @@ -43,7 +44,7 @@ class BrokerApiVersionsCommandTest extends KafkaServerTestHarness { TestUtils.createBrokerConfigs(1, null).map(props => { // Enable unstable api versions to be compatible with the new APIs under development, // maybe we can remove this after the new APIs is complete. - props.setProperty(KafkaConfig.UnstableApiVersionsEnableProp, "true") + props.setProperty(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") props }).map(KafkaConfig.fromProps) } else { @@ -54,7 +55,7 @@ class BrokerApiVersionsCommandTest extends KafkaServerTestHarness { props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT") props.setProperty("listeners", "PLAINTEXT://localhost:0,CONTROLLER://localhost:0") props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "PLAINTEXT://localhost:0,CONTROLLER://localhost:0") - props.setProperty(KafkaConfig.UnstableApiVersionsEnableProp, "true") + props.setProperty(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") props }).map(KafkaConfig.fromProps) } diff --git a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala index e5c2d08e0d1ca..e45689b67f02c 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala @@ -13,7 +13,7 @@ package kafka.api import kafka.security.authorizer.AclAuthorizer -import kafka.server.{BaseRequestTest, KafkaConfig} +import kafka.server.BaseRequestTest import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.ProducerConfig @@ -31,6 +31,7 @@ import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuild import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.metadata.authorizer.StandardAuthorizer +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.{BeforeEach, TestInfo} import java.util.Properties @@ -93,7 +94,7 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest { consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, group) override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) + properties.put(ServerConfigs.BROKER_ID_CONFIG, brokerId.toString) addNodeProperties(properties) } @@ -105,10 +106,10 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest { private def addNodeProperties(properties: Properties): Unit = { if (isKRaftTest()) { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) properties.put(StandardAuthorizer.SUPER_USERS_CONFIG, BrokerPrincipal.toString) } else { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getName) } properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") @@ -116,7 +117,7 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest { properties.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, "1") properties.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, "1") properties.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, "1") - properties.put(KafkaConfig.UnstableApiVersionsEnableProp, "true") + properties.put(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[PrincipalBuilder].getName) } diff --git a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala index b08b25a8adfdd..d242ea105e665 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala @@ -24,7 +24,7 @@ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.TopicPartition import kafka.utils.TestUtils -import kafka.server.{BaseRequestTest, KafkaConfig} +import kafka.server.BaseRequestTest import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, TestInfo} @@ -33,6 +33,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.common.errors.WakeupException import org.apache.kafka.coordinator.group.GroupCoordinatorConfig +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.util.ShutdownableThread import scala.collection.mutable @@ -66,7 +67,7 @@ abstract class AbstractConsumerTest extends BaseRequestTest { override protected def brokerPropertyOverrides(properties: Properties): Unit = { - properties.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown + properties.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") // speed up shutdown properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "3") // don't want to lose offset properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, "100") // set small enough session timeout diff --git a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala index 30b0b55f349ca..3e99a0316d2dd 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala @@ -21,11 +21,11 @@ import kafka.utils.TestUtils.assertFutureExceptionTypeEquals import kafka.utils.{Logging, TestUtils} 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.config.{ConfigResource, SslConfigs, TopicConfig} import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, PolicyViolationException} import org.apache.kafka.common.utils.Utils import org.apache.kafka.network.SocketServerConfigs -import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ServerLogConfigs} import org.apache.kafka.server.policy.AlterConfigPolicy import org.apache.kafka.storage.internals.log.LogConfig import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue} @@ -127,10 +127,10 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with // 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 brokerConfigs = Seq(new ConfigEntry(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "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) + assertEquals(Set(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG), validationsForResource(brokerResource).head.configs().keySet().asScala) validations.clear() val topicConfigEntries1 = Seq( @@ -142,7 +142,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with val topicConfigEntries3 = Seq(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "-1")).asJava - val brokerConfigEntries = Seq(new ConfigEntry(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "12313")).asJava + val brokerConfigEntries = Seq(new ConfigEntry(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "12313")).asJava // Alter configs: second is valid, the others are invalid var alterResult = client.alterConfigs(Map( @@ -172,7 +172,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with assertEquals("0.8", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) - assertNull(configs.get(brokerResource).get(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value) + assertNull(configs.get(brokerResource).get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value) // Alter configs with validateOnly = true: only second is valid topicConfigEntries2 = Seq(new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "0.7")).asJava @@ -204,7 +204,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with assertEquals("0.8", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) - assertNull(configs.get(brokerResource).get(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value) + assertNull(configs.get(brokerResource).get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).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( diff --git a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala index bcfa7f2aa6c24..c6dd0150e25cf 100644 --- a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala @@ -19,19 +19,18 @@ package kafka.api import java.util import java.util.Properties import java.util.concurrent.ExecutionException -import kafka.server.KafkaConfig import kafka.utils.Logging import kafka.utils.TestUtils._ import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateTopicsOptions, CreateTopicsResult, DescribeClusterOptions, DescribeTopicsOptions, NewTopic, TopicDescription} import org.apache.kafka.common.Uuid import org.apache.kafka.common.acl.AclOperation +import org.apache.kafka.common.config.SslConfigs import org.apache.kafka.common.errors.{TopicExistsException, UnknownTopicOrPartitionException} import org.apache.kafka.common.resource.ResourceType import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.KafkaSecurityConfigs -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout} @@ -71,7 +70,7 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg @Test def testCreateDeleteTopics(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topics = Seq("mytopic", "mytopic2", "mytopic3") val newTopics = Seq( new NewTopic("mytopic", Map((0: Integer) -> Seq[Integer](1, 2).asJava, (1: Integer) -> Seq[Integer](2, 0).asJava).asJava), @@ -163,7 +162,7 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg @Test def testAuthorizedOperations(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // without includeAuthorizedOperations flag var result = client.describeCluster @@ -204,14 +203,14 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg }) } configs.foreach { config => - config.setProperty(KafkaConfig.DeleteTopicEnableProp, "true") + config.setProperty(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "true") config.setProperty(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") config.setProperty(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, "false") - config.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") + config.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") // We set this in order to test that we don't expose sensitive data via describe configs. This will already be // set for subclasses with security enabled and we don't want to overwrite it. - if (!config.containsKey(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)) - config.setProperty(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "some.invalid.pass") + if (!config.containsKey(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)) + config.setProperty(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "some.invalid.pass") } } @@ -226,12 +225,17 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg val config = new util.HashMap[String, Object] config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()) config.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "20000") - val securityProps: util.Map[Object, Object] = - adminClientSecurityConfigs(securityProtocol, trustStoreFile, clientSaslProperties) - securityProps.forEach { (key, value) => config.put(key.asInstanceOf[String], value) } + config.put(AdminClientConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, "40000") config } + def createAdminClient: Admin = { + val props = new Properties() + props.putAll(createConfig) + val client = createAdminClient(configOverrides = props) + client + } + def waitForTopics(client: Admin, expectedPresent: Seq[String], expectedMissing: Seq[String]): Unit = { waitUntilTrue(() => { val topics = client.listTopics.names.get() diff --git a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala index 53429f4183e1e..e27fa687f11dd 100644 --- a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala @@ -20,7 +20,7 @@ import java.util.concurrent.TimeUnit import java.util.{Collections, Properties} import com.yammer.metrics.core.{Histogram, Meter} import kafka.api.QuotaTestClients._ -import kafka.server.{ClientQuotaManager, KafkaBroker, KafkaConfig, QuotaType} +import kafka.server.{ClientQuotaManager, KafkaBroker, QuotaType} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.Admin import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig} @@ -33,7 +33,7 @@ import org.apache.kafka.common.quota.ClientQuotaAlteration import org.apache.kafka.common.quota.ClientQuotaEntity import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.server.config.QuotaConfigs +import org.apache.kafka.server.config.{ServerConfigs, QuotaConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, TestInfo} @@ -51,7 +51,7 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { protected def consumerClientId = "QuotasTestConsumer-1" protected def createQuotaTestClients(topic: String, leaderNode: KafkaBroker): QuotaTestClients - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") + this.serverConfig.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "2") this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") this.serverConfig.setProperty(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, "100") diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala index d01476f999df1..a82aca2f76f9f 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala @@ -21,14 +21,11 @@ import java.lang.{Boolean => JBoolean} import java.time.Duration import java.util import java.util.Collections - -import kafka.server.KafkaConfig import kafka.utils.{EmptyTestInfo, TestUtils} import org.apache.kafka.clients.admin.NewTopic import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} -import org.apache.kafka.server.config.ServerLogConfigs - +import org.apache.kafka.server.config.{ServerConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, MethodSource} @@ -57,7 +54,7 @@ object ConsumerTopicCreationTest { private val consumerClientId = "ConsumerTestConsumer" // configure server properties - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown + this.serverConfig.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") // speed up shutdown this.serverConfig.setProperty(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, brokerAutoTopicCreationEnable.toString) // configure client properties diff --git a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala index 9818b6dbd554d..d6a2d105e8de2 100644 --- a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala +++ b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala @@ -29,9 +29,10 @@ import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.{Cluster, Reconfigurable} import org.apache.kafka.common.config.SaslConfigs +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth._ -import org.apache.kafka.server.config.{KafkaSecurityConfigs, QuotaConfigs} +import org.apache.kafka.server.config.{ServerConfigs, QuotaConfigs} import org.apache.kafka.server.quota._ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -63,9 +64,9 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup { override def setUp(testInfo: TestInfo): Unit = { startSasl(jaasSections(kafkaServerSaslMechanisms, Some("SCRAM-SHA-256"), KafkaSasl, JaasTestUtils.KafkaServerContextName)) this.serverConfig.setProperty(QuotaConfigs.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, classOf[GroupedUserQuotaCallback].getName) - this.serverConfig.setProperty(s"${listenerName.configPrefix}${KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG}", + this.serverConfig.setProperty(s"${listenerName.configPrefix}${BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG}", classOf[GroupedUserPrincipalBuilder].getName) - this.serverConfig.setProperty(KafkaConfig.DeleteTopicEnableProp, "true") + this.serverConfig.setProperty(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "true") super.setUp(testInfo) producerConfig.put(SaslConfigs.SASL_JAAS_CONFIG, @@ -367,7 +368,7 @@ class GroupedUserQuotaCallback extends ClientQuotaCallback with Reconfigurable w val partitionRatio = new ConcurrentHashMap[String, Double]() override def configure(configs: util.Map[String, _]): Unit = { - brokerId = configs.get(KafkaConfig.BrokerIdProp).toString.toInt + brokerId = configs.get(ServerConfigs.BROKER_ID_CONFIG).toString.toInt callbackInstances.incrementAndGet } diff --git a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala index a36f54abdc682..95cfc201cd4bd 100644 --- a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala @@ -17,8 +17,6 @@ package kafka.api import java.util.Properties - -import kafka.server.KafkaConfig import kafka.utils._ import kafka.tools.StorageTool import kafka.zk.ConfigEntityChangeNotificationZNode @@ -35,6 +33,7 @@ import org.junit.jupiter.api.{BeforeEach, TestInfo} import scala.jdk.CollectionConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.config.DelegationTokenManagerConfigs class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest { @@ -54,8 +53,8 @@ class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest protected val privilegedAdminClientConfig = new Properties() - this.serverConfig.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "testKey") - this.controllerConfig.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "testKey") + this.serverConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "testKey") + this.controllerConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "testKey") def createDelegationTokenOptions(): CreateDelegationTokenOptions = new CreateDelegationTokenOptions() diff --git a/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala b/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala index 0be5e306e2948..c150b70415dfe 100644 --- a/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala +++ b/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala @@ -14,9 +14,7 @@ package kafka.api import java.util import java.util.Properties - import kafka.security.authorizer.AclAuthorizer -import kafka.server.KafkaConfig import kafka.utils.{CoreUtils, JaasTestUtils, TestUtils} import org.apache.kafka.clients.admin._ import org.apache.kafka.common.acl.AclOperation.{ALL, ALTER, CLUSTER_ACTION, DELETE, DESCRIBE} @@ -27,7 +25,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.utils.Utils import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.authorizer.Authorizer -import org.apache.kafka.server.config.ZkConfigs +import org.apache.kafka.server.config.{ServerConfigs, ZkConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNull} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -78,7 +76,7 @@ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslS override val brokerCount = 1 this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true") - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getName) var client: Admin = _ diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala index 60dee252420fb..d32c5b74cc94b 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala @@ -23,7 +23,6 @@ import java.util.{Collections, Properties} import java.util.concurrent.ExecutionException import kafka.security.authorizer.AclAuthorizer import org.apache.kafka.metadata.authorizer.StandardAuthorizer -import kafka.server._ import kafka.utils._ import org.apache.kafka.clients.admin.Admin import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecords} @@ -31,6 +30,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, Produce import org.apache.kafka.common.acl._ import org.apache.kafka.common.acl.AclOperation._ import org.apache.kafka.common.acl.AclPermissionType._ +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.{GroupAuthorizationException, TopicAuthorizationException} import org.apache.kafka.common.resource._ @@ -39,8 +39,7 @@ import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.security.auth._ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST -import org.apache.kafka.server.config.{KafkaSecurityConfigs, ZkConfigs} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout} @@ -139,7 +138,7 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "3") this.serverConfig.setProperty(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "3") this.serverConfig.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "3") - this.serverConfig.setProperty(KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, "1500") + this.serverConfig.setProperty(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, "1500") this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "group") this.consumerConfig.setProperty(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1500") @@ -153,13 +152,13 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas if (TestInfoUtils.isKRaft(testInfo)) { this.serverConfig.setProperty(StandardAuthorizer.SUPER_USERS_CONFIG, kafkaPrincipal.toString) this.controllerConfig.setProperty(StandardAuthorizer.SUPER_USERS_CONFIG, kafkaPrincipal.toString + ";" + "User:ANONYMOUS") - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) - this.controllerConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) + this.controllerConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) } else { // The next two configuration parameters enable ZooKeeper secure ACLs // and sets the Kafka authorizer, both necessary to enable security. this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true") - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, authorizerClass.getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, authorizerClass.getName) // Set the specific principal that can update ACLs. this.serverConfig.setProperty(AclAuthorizer.SuperUsersProp, kafkaPrincipal.toString) diff --git a/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala index 6ffd0f7fb19bb..02c8399fa0898 100644 --- a/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala @@ -16,7 +16,7 @@ import java.util.Properties import java.util.concurrent.ExecutionException import kafka.api.GroupAuthorizerIntegrationTest._ import kafka.security.authorizer.AclAuthorizer -import kafka.server.{BaseRequestTest, KafkaConfig} +import kafka.server.BaseRequestTest import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.ProducerRecord @@ -32,6 +32,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.metadata.authorizer.StandardAuthorizer import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -76,16 +77,16 @@ class GroupAuthorizerIntegrationTest extends BaseRequestTest { } override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) + properties.put(ServerConfigs.BROKER_ID_CONFIG, brokerId.toString) addNodeProperties(properties) } private def addNodeProperties(properties: Properties): Unit = { if (isKRaftTest()) { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) properties.put(StandardAuthorizer.SUPER_USERS_CONFIG, BrokerPrincipal.toString) } else { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getName) } properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index e1f549c4e5357..192dfa392d907 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -62,7 +62,6 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { } override def generateConfigs: Seq[KafkaConfig] = { - val cfgs = TestUtils.createBrokerConfigs(brokerCount, zkConnectOrNull, interBrokerSecurityProtocol = Some(securityProtocol), trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties, logDirCount = logDirCount) configureListeners(cfgs) @@ -72,6 +71,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { } if (isNewGroupCoordinatorEnabled()) { cfgs.foreach(_.setProperty(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true")) + cfgs.foreach(_.setProperty(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer")) } if(isKRaftTest()) { diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 2133019a4a871..ffe10c19c4392 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -36,7 +36,7 @@ import org.apache.kafka.clients.admin._ import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclBindingFilter, AclOperation, AclPermissionType} -import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, TopicConfig} +import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, SslConfigs, TopicConfig} import org.apache.kafka.common.errors._ import org.apache.kafka.common.requests.{DeleteRecordsRequest, MetadataResponse} import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType} @@ -46,7 +46,7 @@ import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEX import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.{KafkaSecurityConfigs, QuotaConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{QuotaConfigs, ServerConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS import org.junit.jupiter.api.Assertions._ @@ -95,7 +95,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testClose(quorum: String): Unit = { - val client = Admin.create(createConfig) + val client = createAdminClient client.close() client.close() // double close has no effect } @@ -103,7 +103,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testListNodes(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val brokerStrs = bootstrapServers().split(",").toList.sorted var nodeStrs: List[String] = null do { @@ -131,7 +131,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testCreateExistingTopicsThrowTopicExistsException(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topic = "mytopic" val topics = Seq(topic) val newTopics = Seq(new NewTopic(topic, 1, 1.toShort)) @@ -148,7 +148,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDeleteTopicsWithIds(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topics = Seq("mytopic", "mytopic2", "mytopic3") val newTopics = Seq( new NewTopic("mytopic", Map((0: Integer) -> Seq[Integer](1, 2).asJava, (1: Integer) -> Seq[Integer](2, 0).asJava).asJava), @@ -170,7 +170,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeNonExistingTopic(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val existingTopic = "existing-topic" client.createTopics(Seq(existingTopic).map(new NewTopic(_, 1, 1.toShort)).asJava).all.get() @@ -188,7 +188,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeTopicsWithIds(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val existingTopic = "existing-topic" client.createTopics(Seq(existingTopic).map(new NewTopic(_, 1, 1.toShort)).asJava).all.get() @@ -207,7 +207,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeCluster(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val result = client.describeCluster val nodes = result.nodes.get() val clusterId = result.clusterId().get() @@ -234,7 +234,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeLogDirs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topic = "topic" val leaderByPartition = createTopic(topic, numPartitions = 10) val partitionsByBroker = leaderByPartition.groupBy { case (_, leaderId) => leaderId }.map { case (k, v) => @@ -266,7 +266,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeReplicaLogDirs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topic = "topic" val leaderByPartition = createTopic(topic, numPartitions = 10) val replicas = leaderByPartition.map { case (partition, brokerId) => @@ -285,7 +285,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testAlterReplicaLogDirs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topic = "topic" val tp = new TopicPartition(topic, 0) val randomNums = brokers.map(server => server -> Random.nextInt(2)).toMap @@ -375,7 +375,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeAndAlterConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Create topics val topic1 = "describe-alter-configs-topic-1" @@ -426,29 +426,29 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { }) assertEquals(brokers(1).config.nonInternalValues.size + numInternalConfigsSet, configs.get(brokerResource1).entries.size) - assertEquals(brokers(1).config.brokerId.toString, configs.get(brokerResource1).get(KafkaConfig.BrokerIdProp).value) + assertEquals(brokers(1).config.brokerId.toString, configs.get(brokerResource1).get(ServerConfigs.BROKER_ID_CONFIG).value) val listenerSecurityProtocolMap = configs.get(brokerResource1).get(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG) assertEquals(brokers(1).config.getString(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG), listenerSecurityProtocolMap.value) assertEquals(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, listenerSecurityProtocolMap.name) assertFalse(listenerSecurityProtocolMap.isDefault) assertFalse(listenerSecurityProtocolMap.isSensitive) assertFalse(listenerSecurityProtocolMap.isReadOnly) - val truststorePassword = configs.get(brokerResource1).get(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG) - assertEquals(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, truststorePassword.name) + val truststorePassword = configs.get(brokerResource1).get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG) + assertEquals(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, truststorePassword.name) assertNull(truststorePassword.value) assertFalse(truststorePassword.isDefault) assertTrue(truststorePassword.isSensitive) assertFalse(truststorePassword.isReadOnly) - val compressionType = configs.get(brokerResource1).get(KafkaConfig.CompressionTypeProp) + val compressionType = configs.get(brokerResource1).get(ServerConfigs.COMPRESSION_TYPE_CONFIG) assertEquals(brokers(1).config.compressionType, compressionType.value) - assertEquals(KafkaConfig.CompressionTypeProp, compressionType.name) + assertEquals(ServerConfigs.COMPRESSION_TYPE_CONFIG, compressionType.name) assertTrue(compressionType.isDefault) assertFalse(compressionType.isSensitive) assertFalse(compressionType.isReadOnly) assertEquals(brokers(2).config.nonInternalValues.size + numInternalConfigsSet, configs.get(brokerResource2).entries.size) - assertEquals(brokers(2).config.brokerId.toString, configs.get(brokerResource2).get(KafkaConfig.BrokerIdProp).value) + assertEquals(brokers(2).config.brokerId.toString, configs.get(brokerResource2).get(ServerConfigs.BROKER_ID_CONFIG).value) assertEquals(brokers(2).config.logCleanerThreads.toString, configs.get(brokerResource2).get(CleanerConfig.LOG_CLEANER_THREADS_PROP).value) @@ -458,7 +458,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testCreatePartitions(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Create topics val topic1 = "create-partitions-topic-1" @@ -731,7 +731,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { def testSeekAfterDeleteRecords(quorum: String): Unit = { createTopic(topic, numPartitions = 2, replicationFactor = brokerCount) - client = Admin.create(createConfig) + client = createAdminClient val consumer = createConsumer() subscribeAndWaitForAssignment(topic, consumer) @@ -761,7 +761,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { def testLogStartOffsetCheckpoint(quorum: String): Unit = { createTopic(topic, numPartitions = 2, replicationFactor = brokerCount) - client = Admin.create(createConfig) + client = createAdminClient val consumer = createConsumer() subscribeAndWaitForAssignment(topic, consumer) @@ -778,7 +778,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { restartDeadBrokers() client.close() - client = Admin.create(createConfig) + client = createAdminClient TestUtils.waitUntilTrue(() => { // Need to retry if leader is not available for the partition @@ -801,7 +801,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { def testLogStartOffsetAfterDeleteRecords(quorum: String): Unit = { createTopic(topic, numPartitions = 2, replicationFactor = brokerCount) - client = Admin.create(createConfig) + client = createAdminClient val consumer = createConsumer() subscribeAndWaitForAssignment(topic, consumer) @@ -840,7 +840,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { // we will produce to topic and delete records while one follower is down killBroker(followerIndex) - client = Admin.create(createConfig) + client = createAdminClient val producer = createProducer() sendRecords(producer, 100, topicPartition) @@ -869,7 +869,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testAlterLogDirsAfterDeleteRecords(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient createTopic(topic, replicationFactor = brokerCount) val expectedLEO = 100 val producer = createProducer() @@ -904,7 +904,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { def testOffsetsForTimesAfterDeleteRecords(quorum: String): Unit = { createTopic(topic, numPartitions = 2, replicationFactor = brokerCount) - client = Admin.create(createConfig) + client = createAdminClient val consumer = createConsumer() subscribeAndWaitForAssignment(topic, consumer) @@ -928,7 +928,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val consumer = createConsumer() subscribeAndWaitForAssignment(topic, consumer) - client = Admin.create(createConfig) + client = createAdminClient val producer = createProducer() sendRecords(producer, 10, topicPartition) @@ -952,7 +952,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val consumer = createConsumer() subscribeAndWaitForAssignment(topic, consumer) - client = Admin.create(createConfig) + client = createAdminClient val producer = createProducer() sendRecords(producer, 10, topicPartition) @@ -970,7 +970,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testDescribeConfigsForTopic(quorum: String): Unit = { createTopic(topic, numPartitions = 2, replicationFactor = brokerCount) - client = Admin.create(createConfig) + client = createAdminClient val existingTopic = new ConfigResource(ConfigResource.Type.TOPIC, topic) client.describeConfigs(Collections.singletonList(existingTopic)).values.get(existingTopic).get() @@ -1006,7 +1006,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testInvalidAlterConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient checkInvalidAlterConfigs(this, client) } @@ -1020,7 +1020,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { def testAclOperations(quorum: String): Unit = { val acl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic3", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.ALLOW)) - client = Admin.create(createConfig) + client = createAdminClient assertFutureExceptionTypeEquals(client.describeAcls(AclBindingFilter.ANY).values(), classOf[SecurityDisabledException]) assertFutureExceptionTypeEquals(client.createAcls(Collections.singleton(acl)).all(), classOf[SecurityDisabledException]) @@ -1035,7 +1035,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDelayedClose(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topics = Seq("mytopic", "mytopic2") val newTopics = topics.map(new NewTopic(_, 1, 1.toShort)) val future = client.createTopics(newTopics.asJava, new CreateTopicsOptions().validateOnly(true)).all() @@ -1400,7 +1400,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testElectPreferredLeaders(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val prefer0 = Seq(0, 1, 2) val prefer1 = Seq(1, 2, 0) @@ -1560,7 +1560,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersForOnePartition(quorum: String): Unit = { // Case: unclean leader election with one topic partition - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1588,7 +1588,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersForManyPartitions(quorum: String): Unit = { // Case: unclean leader election with many topic partitions - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1628,7 +1628,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersForAllPartitions(quorum: String): Unit = { // Case: noop unclean leader election and valid unclean leader election for all partitions - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1668,7 +1668,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersForUnknownPartitions(quorum: String): Unit = { // Case: unclean leader election for unknown topic - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1694,7 +1694,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersWhenNoLiveBrokers(quorum: String): Unit = { // Case: unclean leader election with no live brokers - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1723,7 +1723,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersNoop(quorum: String): Unit = { // Case: noop unclean leader election with explicit topic partitions - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1751,7 +1751,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersAndNoop(quorum: String): Unit = { // Case: one noop unclean leader election and one valid unclean leader election - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1790,7 +1790,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testListReassignmentsDoesNotShowNonReassigningPartitions(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Create topics val topic = "list-reassignments-no-reassignments" @@ -1807,7 +1807,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testListReassignmentsDoesNotShowDeletedPartitions(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topic = "list-reassignments-no-reassignments" val tp = new TopicPartition(topic, 0) @@ -1822,7 +1822,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testValidIncrementalAlterConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Create topics val topic1 = "incremental-alter-configs-topic-1" @@ -1942,7 +1942,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testAppendAlreadyExistsConfigsAndSubtractNotExistsConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Create topics val topic = "incremental-alter-configs-topic" @@ -1983,7 +1983,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testIncrementalAlterConfigsDeleteAndSetBrokerConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val broker0Resource = new ConfigResource(ConfigResource.Type.BROKER, "0") client.incrementalAlterConfigs(Map(broker0Resource -> Seq(new AlterConfigOp(new ConfigEntry(QuotaConfigs.LEADER_REPLICATION_THROTTLED_RATE_CONFIG, "123"), @@ -2017,7 +2017,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testIncrementalAlterConfigsDeleteBrokerConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val broker0Resource = new ConfigResource(ConfigResource.Type.BROKER, "0") client.incrementalAlterConfigs(Map(broker0Resource -> Seq(new AlterConfigOp(new ConfigEntry(QuotaConfigs.LEADER_REPLICATION_THROTTLED_RATE_CONFIG, "123"), @@ -2054,7 +2054,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testInvalidIncrementalAlterConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Create topics val topic1 = "incremental-alter-configs-topic-1" @@ -2150,7 +2150,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testInvalidAlterPartitionReassignments(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topic = "alter-reassignments-topic-1" val tp1 = new TopicPartition(topic, 0) val tp2 = new TopicPartition(topic, 1) @@ -2190,9 +2190,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testLongTopicNames(quorum: String): Unit = { - val client = Admin.create(createConfig) - val longTopicName = List.fill(249)("x").mkString("") - val invalidTopicName = List.fill(250)("x").mkString("") + val client = createAdminClient + val longTopicName = String.join("", Collections.nCopies(249, "x")); + val invalidTopicName = String.join("", Collections.nCopies(250, "x")); val newTopics2 = Seq(new NewTopic(invalidTopicName, 3, 3.toShort), new NewTopic(longTopicName, 3, 3.toShort)) val results = client.createTopics(newTopics2.asJava).values() @@ -2225,7 +2225,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { assertEquals(ServerLogConfigs.LOG_RETENTION_BYTES_DEFAULT, logConfig.retentionSize) } - client = Admin.create(createConfig) + client = createAdminClient val invalidConfigs = Map[String, String]( TopicConfig.RETENTION_BYTES_CONFIG -> null, TopicConfig.COMPRESSION_TYPE_CONFIG -> "producer" @@ -2258,7 +2258,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeConfigsForLog4jLogLevels(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient LoggerFactory.getLogger("kafka.cluster.Replica").trace("Message to create the logger") val loggerConfig = describeBrokerLoggers() val kafkaLogLevel = loggerConfig.get("kafka").value() @@ -2276,7 +2276,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) @Disabled // To be re-enabled once KAFKA-8779 is resolved def testIncrementalAlterConfigsForLog4jLogLevels(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val initialLoggerConfig = describeBrokerLoggers() val initialRootLogLevel = initialLoggerConfig.get(Log4jController.ROOT_LOGGER).value() @@ -2341,7 +2341,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) @Disabled // To be re-enabled once KAFKA-8779 is resolved def testIncrementalAlterConfigsForLog4jLogLevelsCanResetLoggerToCurrentRoot(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // step 1 - configure root logger val initialRootLogLevel = LogLevelConfig.TRACE_LOG_LEVEL val alterRootLoggerEntry = Seq( @@ -2384,7 +2384,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) @Disabled // Zk to be re-enabled once KAFKA-8779 is resolved def testIncrementalAlterConfigsForLog4jLogLevelsCannotResetRootLogger(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val deleteRootLoggerEntry = Seq( new AlterConfigOp(new ConfigEntry(Log4jController.ROOT_LOGGER, ""), AlterConfigOp.OpType.DELETE) ).asJavaCollection @@ -2396,7 +2396,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) @Disabled // To be re-enabled once KAFKA-8779 is resolved def testIncrementalAlterConfigsForLog4jLogLevelsDoesNotWorkWithInvalidConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val validLoggerName = "kafka.server.KafkaRequestHandler" val expectedValidLoggerLogLevel = describeBrokerLoggers().get(validLoggerName) def assertLogLevelDidNotChange(): Unit = { @@ -2440,7 +2440,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("kraft")) // Zk to be re-enabled once KAFKA-8779 is resolved def testAlterConfigsForLog4jLogLevelsDoesNotWork(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val alterLogLevelsEntries = Seq( new ConfigEntry("kafka.controller.KafkaController", LogLevelConfig.INFO_LOG_LEVEL) @@ -2499,7 +2499,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { } private def testAppendConfig(props: Properties, append: String, expected: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient createTopic(topic, topicConfig = props) val topicResource = new ConfigResource(ConfigResource.Type.TOPIC, topic) val topicAlterConfigs = Seq( @@ -2735,7 +2735,7 @@ object PlaintextAdminIntegrationTest { assertEquals("snappy", configs.get(topicResource2).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value) - assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value) + assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(ServerConfigs.COMPRESSION_TYPE_CONFIG).value) // Alter configs with validateOnly = true: first and third are invalid, second is valid topicConfigEntries2 = Seq(new ConfigEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, "gzip")).asJava @@ -2764,6 +2764,6 @@ object PlaintextAdminIntegrationTest { assertEquals("snappy", configs.get(topicResource2).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value) - assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value) + assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(ServerConfigs.COMPRESSION_TYPE_CONFIG).value) } } diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 71cd632c0d8f0..a05f3a20f1eba 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -44,8 +44,8 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { val overridingProps = new Properties() overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString) - overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) - overridingProps.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, replicaFetchMaxPartitionBytes.toString) + overridingProps.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, serverMessageMaxBytes.toString) + overridingProps.put(ReplicationConfigs.REPLICA_FETCH_MAX_BYTES_CONFIG, replicaFetchMaxPartitionBytes.toString) overridingProps.put(ReplicationConfigs.REPLICA_FETCH_RESPONSE_MAX_BYTES_DOC, replicaFetchMaxResponseBytes.toString) // Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic) // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long diff --git a/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala b/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala index b15e9f9f2d501..1b6a99afb10f4 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors.{InvalidPidMappingException, TransactionalIdNotFoundException} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.test.{TestUtils => JTestUtils} import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} @@ -233,7 +233,7 @@ class ProducerIdExpirationTest extends KafkaServerTestHarness { serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 3.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, 2.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, 2.toString) - serverProps.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + serverProps.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) serverProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) serverProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) serverProps.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") diff --git a/core/src/test/scala/integration/kafka/api/SaslGssapiSslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SaslGssapiSslEndToEndAuthorizationTest.scala index 7480dce44e04a..e1f1638fa0a61 100644 --- a/core/src/test/scala/integration/kafka/api/SaslGssapiSslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslGssapiSslEndToEndAuthorizationTest.scala @@ -19,9 +19,8 @@ package kafka.api import kafka.security.authorizer.AclAuthorizer import kafka.utils.JaasTestUtils import org.apache.kafka.common.config.SslConfigs +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.security.auth._ -import org.apache.kafka.server.config.KafkaSecurityConfigs - import org.junit.jupiter.api.Assertions.assertNull import scala.collection.immutable.List @@ -39,8 +38,8 @@ class SaslGssapiSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTe // Configure brokers to require SSL client authentication in order to verify that SASL_SSL works correctly even if the // client doesn't have a keystore. We want to cover the scenario where a broker requires either SSL client // authentication or SASL authentication with SSL as the transport layer (but not both). - serverConfig.put(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required") - controllerConfig.put(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required") + serverConfig.put(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required") + controllerConfig.put(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required") assertNull(producerConfig.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) assertNull(consumerConfig.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) assertNull(adminClientConfig.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) diff --git a/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala index b95e39a3e9363..0ae3c437cf649 100644 --- a/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala @@ -26,7 +26,6 @@ import org.apache.kafka.common.network.Mode import org.apache.kafka.common.security.auth._ import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder import org.apache.kafka.common.security.plain.PlainAuthenticateCallback -import org.apache.kafka.server.config.KafkaSecurityConfigs import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.Test @@ -111,11 +110,11 @@ object SaslPlainSslEndToEndAuthorizationTest { class SaslPlainSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTest { import SaslPlainSslEndToEndAuthorizationTest._ - this.serverConfig.setProperty(s"${listenerName.configPrefix}${KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG}", "required") + this.serverConfig.setProperty(s"${listenerName.configPrefix}${BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG}", "required") this.serverConfig.setProperty(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[TestPrincipalBuilder].getName) - this.serverConfig.put(KafkaSecurityConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_CONFIG, classOf[TestClientCallbackHandler].getName) + this.serverConfig.put(SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS, classOf[TestClientCallbackHandler].getName) val mechanismPrefix = listenerName.saslMechanismConfigPrefix("PLAIN") - this.serverConfig.put(s"$mechanismPrefix${KafkaSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG}", classOf[TestServerCallbackHandler].getName) + this.serverConfig.put(s"$mechanismPrefix${BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG}", classOf[TestServerCallbackHandler].getName) this.producerConfig.put(SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS, classOf[TestClientCallbackHandler].getName) this.consumerConfig.put(SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS, classOf[TestClientCallbackHandler].getName) this.adminClientConfig.put(SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS, classOf[TestClientCallbackHandler].getName) diff --git a/core/src/test/scala/integration/kafka/api/SaslSetup.scala b/core/src/test/scala/integration/kafka/api/SaslSetup.scala index b00ddad3246a3..5a9bd4b9db201 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSetup.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSetup.scala @@ -35,7 +35,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.authenticator.LoginManager import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter, ScramMechanism} import org.apache.kafka.common.utils.Time -import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs} +import org.apache.kafka.server.config.ConfigType import org.apache.zookeeper.client.ZKClientConfig import scala.util.Using @@ -133,7 +133,7 @@ trait SaslSetup { def kafkaServerSaslProperties(serverSaslMechanisms: Seq[String], interBrokerSaslMechanism: String): Properties = { val props = new Properties - props.put(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, interBrokerSaslMechanism) + props.put(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, interBrokerSaslMechanism) props.put(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, serverSaslMechanisms.mkString(",")) props } diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 47e8de08031b4..1fc4c3e152a4f 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -12,63 +12,87 @@ */ package kafka.api -import java.util import kafka.security.authorizer.AclAuthorizer -import kafka.server.KafkaConfig -import kafka.utils.{CoreUtils, JaasTestUtils, TestUtils} import kafka.utils.TestUtils._ +import kafka.utils.{JaasTestUtils, TestUtils} import org.apache.kafka.clients.admin._ import org.apache.kafka.common.Uuid import org.apache.kafka.common.acl._ -import org.apache.kafka.common.acl.AclOperation.{ALL, ALTER, ALTER_CONFIGS, CLUSTER_ACTION, CREATE, DELETE, DESCRIBE} +import org.apache.kafka.common.acl.AclOperation.{ALL, ALTER, ALTER_CONFIGS, CLUSTER_ACTION, CREATE, DELETE, DESCRIBE, IDEMPOTENT_WRITE} import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY} -import org.apache.kafka.common.config.{ConfigResource, TopicConfig} +import org.apache.kafka.common.config.{ConfigResource, SaslConfigs, TopicConfig} import org.apache.kafka.common.errors.{ClusterAuthorizationException, InvalidRequestException, TopicAuthorizationException, UnknownTopicOrPartitionException} import org.apache.kafka.common.resource.PatternType.LITERAL import org.apache.kafka.common.resource.ResourceType.{GROUP, TOPIC} import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.security.authorizer.AclEntry.{WILDCARD_HOST, WILDCARD_PRINCIPAL_STRING} -import org.apache.kafka.server.authorizer.Authorizer -import org.apache.kafka.server.config.ZkConfigs +import org.apache.kafka.server.config.{ServerConfigs, ZkConfigs} import org.apache.kafka.storage.internals.log.LogConfig import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout} -import java.util.Collections -import scala.jdk.CollectionConverters._ +import java.util import scala.collection.Seq import scala.compat.java8.OptionConverters._ import scala.concurrent.ExecutionException +import scala.jdk.CollectionConverters._ import scala.util.{Failure, Success, Try} +@Timeout(120) class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetup { val clusterResourcePattern = new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL) - val authorizationAdmin = new AclAuthorizationAdmin(classOf[AclAuthorizer], classOf[AclAuthorizer]) - - this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true") + val aclAuthorizerClassName: String = classOf[AclAuthorizer].getName + def kafkaPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, JaasTestUtils.KafkaServerPrincipalUnqualifiedName) + var superUserAdmin: Admin = _ override protected def securityProtocol = SecurityProtocol.SASL_SSL override protected lazy val trustStoreFile = Some(TestUtils.tempFile("truststore", ".jks")) - override def generateConfigs: Seq[KafkaConfig] = { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, authorizationAdmin.authorizerClassName) - super.generateConfigs - } - - override def configureSecurityBeforeServersStart(testInfo: TestInfo): Unit = { - authorizationAdmin.initializeAcls() - } - @BeforeEach override def setUp(testInfo: TestInfo): Unit = { + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, aclAuthorizerClassName) + this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true") + this.serverConfig.setProperty(AclAuthorizer.SuperUsersProp, kafkaPrincipal.toString) + setUpSasl() super.setUp(testInfo) + setInitialAcls() } def setUpSasl(): Unit = { startSasl(jaasSections(Seq("GSSAPI"), Some("GSSAPI"), Both, JaasTestUtils.KafkaServerContextName)) + + val loginContext = jaasAdminLoginModule("GSSAPI") + superuserClientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, loginContext) + } + + private def setInitialAcls(): Unit = { + superUserAdmin = createSuperuserAdminClient() + val ace = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, ALL, ALLOW) + superUserAdmin.createAcls(List(new AclBinding(new ResourcePattern(TOPIC, "*", LITERAL), ace)).asJava) + superUserAdmin.createAcls(List(new AclBinding(new ResourcePattern(GROUP, "*", LITERAL), ace)).asJava) + + val clusterAcls = List(clusterAcl(ALLOW, CREATE), + clusterAcl(ALLOW, DELETE), + clusterAcl(ALLOW, CLUSTER_ACTION), + clusterAcl(ALLOW, ALTER_CONFIGS), + clusterAcl(ALLOW, ALTER), + clusterAcl(ALLOW, IDEMPOTENT_WRITE)) + + superUserAdmin.createAcls(clusterAcls.map(ace => new AclBinding(clusterResourcePattern, ace)).asJava) + + brokers.foreach { b => + TestUtils.waitAndVerifyAcls(Set(ace), b.dataPlaneRequestProcessor.authorizer.get, new ResourcePattern(TOPIC, "*", LITERAL)) + TestUtils.waitAndVerifyAcls(Set(ace), b.dataPlaneRequestProcessor.authorizer.get, new ResourcePattern(GROUP, "*", LITERAL)) + TestUtils.waitAndVerifyAcls(clusterAcls.toSet, b.dataPlaneRequestProcessor.authorizer.get, clusterResourcePattern) + } + } + + private def clusterAcl(permissionType: AclPermissionType, operation: AclOperation): AccessControlEntry = { + new AccessControlEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*").toString, + WILDCARD_HOST, operation, permissionType) } @AfterEach @@ -94,10 +118,10 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu @Test def testAclOperations(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val acl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic3", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.ALLOW)) - assertEquals(7, getAcls(AclBindingFilter.ANY).size) + assertEquals(8, getAcls(AclBindingFilter.ANY).size) val results = client.createAcls(List(acl2, acl3).asJava) assertEquals(Set(acl2, acl3), results.values.keySet().asScala) results.values.values.forEach(value => value.get) @@ -115,7 +139,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu @Test def testAclOperations2(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val results = client.createAcls(List(acl2, acl2, transactionalIdAcl).asJava) assertEquals(Set(acl2, acl2, transactionalIdAcl), results.values.keySet.asScala) results.all.get() @@ -141,7 +165,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu @Test def testAclDescribe(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient ensureAcls(Set(anyAcl, acl2, fooAcl, prefixAcl)) val allTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.ANY), AccessControlEntryFilter.ANY) @@ -168,7 +192,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu @Test def testAclDelete(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient ensureAcls(Set(anyAcl, acl2, fooAcl, prefixAcl)) val allTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.MATCH), AccessControlEntryFilter.ANY) @@ -218,7 +242,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu //noinspection ScalaDeprecation - test explicitly covers clients using legacy / deprecated constructors @Test def testLegacyAclOpsNeverAffectOrReturnPrefixed(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient ensureAcls(Set(anyAcl, acl2, fooAcl, prefixAcl)) // <-- prefixed exists, but should never be returned. val allTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.MATCH), AccessControlEntryFilter.ANY) @@ -255,7 +279,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu @Test def testAttemptToCreateInvalidAcls(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val clusterAcl = new AclBinding(new ResourcePattern(ResourceType.CLUSTER, "foobar", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.READ, AclPermissionType.ALLOW)) val emptyResourceNameAcl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "", PatternType.LITERAL), @@ -266,11 +290,6 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu assertFutureExceptionTypeEquals(results.values.get(emptyResourceNameAcl), classOf[InvalidRequestException]) } - override def configuredClusterPermissions: Set[AclOperation] = { - Set(AclOperation.ALTER, AclOperation.CREATE, AclOperation.CLUSTER_ACTION, AclOperation.ALTER_CONFIGS, - AclOperation.DESCRIBE, AclOperation.DESCRIBE_CONFIGS) - } - private def verifyCauseIsClusterAuth(e: Throwable): Unit = assertEquals(classOf[ClusterAuthorizationException], e.getCause.getClass) private def testAclCreateGetDelete(expectAuth: Boolean): Unit = { @@ -350,36 +369,53 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu @Test def testAclAuthorizationDenied(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Test that we cannot create or delete ACLs when ALTER is denied. - authorizationAdmin.addClusterAcl(DENY, ALTER) + addClusterAcl(DENY, ALTER) testAclGet(expectAuth = true) testAclCreateGetDelete(expectAuth = false) // Test that we cannot do anything with ACLs when DESCRIBE and ALTER are denied. - authorizationAdmin.addClusterAcl(DENY, DESCRIBE) + addClusterAcl(DENY, DESCRIBE) testAclGet(expectAuth = false) testAclCreateGetDelete(expectAuth = false) // Test that we can create, delete, and get ACLs with the default ACLs. - authorizationAdmin.removeClusterAcl(DENY, DESCRIBE) - authorizationAdmin.removeClusterAcl(DENY, ALTER) + removeClusterAcl(DENY, DESCRIBE) + removeClusterAcl(DENY, ALTER) testAclGet(expectAuth = true) testAclCreateGetDelete(expectAuth = true) // Test that we can't do anything with ACLs without the ALLOW ALTER ACL in place. - authorizationAdmin.removeClusterAcl(ALLOW, ALTER) - authorizationAdmin.removeClusterAcl(ALLOW, DELETE) + removeClusterAcl(ALLOW, ALTER) + removeClusterAcl(ALLOW, DELETE) testAclGet(expectAuth = false) testAclCreateGetDelete(expectAuth = false) // Test that we can describe, but not alter ACLs, with only the ALLOW DESCRIBE ACL in place. - authorizationAdmin.addClusterAcl(ALLOW, DESCRIBE) + addClusterAcl(ALLOW, DESCRIBE) testAclGet(expectAuth = true) testAclCreateGetDelete(expectAuth = false) } + private def addClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit = { + val ace = clusterAcl(permissionType, operation) + superUserAdmin.createAcls(List(new AclBinding(clusterResourcePattern, ace)).asJava) + brokers.foreach { b => + TestUtils.waitAndVerifyAcl(ace, b.dataPlaneRequestProcessor.authorizer.get, clusterResourcePattern) + } + } + + private def removeClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit = { + val ace = clusterAcl(permissionType, operation) + superUserAdmin.deleteAcls(List(new AclBinding(clusterResourcePattern, ace).toFilter).asJava).values + + brokers.foreach { b => + TestUtils.waitAndVerifyRemovedAcl(ace, b.dataPlaneRequestProcessor.authorizer.get, clusterResourcePattern) + } + } + @Test def testCreateTopicsResponseMetadataAndConfig(): Unit = { val topic1 = "mytopic1" @@ -387,7 +423,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu val denyAcl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, topic2, PatternType.LITERAL), new AccessControlEntry("User:*", "*", AclOperation.DESCRIBE_CONFIGS, AclPermissionType.DENY)) - client = Admin.create(createConfig) + client = createAdminClient client.createAcls(List(denyAcl).asJava, new CreateAclsOptions()).all().get() val topics = Seq(topic1, topic2) @@ -474,54 +510,4 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu private def getAcls(allTopicAcls: AclBindingFilter) = { client.describeAcls(allTopicAcls).values.get().asScala.toSet } - - class AclAuthorizationAdmin(authorizerClass: Class[_ <: AclAuthorizer], authorizerForInitClass: Class[_ <: AclAuthorizer]) { - - def authorizerClassName: String = authorizerClass.getName - - def initializeAcls(): Unit = { - val authorizer = CoreUtils.createObject[Authorizer](authorizerForInitClass.getName) - try { - authorizer.configure(configs.head.originals()) - val ace = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, ALL, ALLOW) - authorizer.createAcls(null, List(new AclBinding(new ResourcePattern(TOPIC, "*", LITERAL), ace)).asJava) - authorizer.createAcls(null, List(new AclBinding(new ResourcePattern(GROUP, "*", LITERAL), ace)).asJava) - - authorizer.createAcls(null, List(clusterAcl(ALLOW, CREATE), - clusterAcl(ALLOW, DELETE), - clusterAcl(ALLOW, CLUSTER_ACTION), - clusterAcl(ALLOW, ALTER_CONFIGS), - clusterAcl(ALLOW, ALTER)) - .map(ace => new AclBinding(clusterResourcePattern, ace)).asJava) - } finally { - authorizer.close() - } - } - - def addClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit = { - val ace = clusterAcl(permissionType, operation) - val aclBinding = new AclBinding(clusterResourcePattern, ace) - val authorizer = servers.head.dataPlaneRequestProcessor.authorizer.get - val prevAcls = authorizer.acls(new AclBindingFilter(clusterResourcePattern.toFilter, AccessControlEntryFilter.ANY)) - .asScala.map(_.entry).toSet - authorizer.createAcls(null, Collections.singletonList(aclBinding)) - TestUtils.waitAndVerifyAcls(prevAcls ++ Set(ace), authorizer, clusterResourcePattern) - } - - def removeClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit = { - val ace = clusterAcl(permissionType, operation) - val authorizer = servers.head.dataPlaneRequestProcessor.authorizer.get - val clusterFilter = new AclBindingFilter(clusterResourcePattern.toFilter, AccessControlEntryFilter.ANY) - val prevAcls = authorizer.acls(clusterFilter).asScala.map(_.entry).toSet - val deleteFilter = new AclBindingFilter(clusterResourcePattern.toFilter, ace.toFilter) - assertFalse(authorizer.deleteAcls(null, Collections.singletonList(deleteFilter)) - .get(0).toCompletableFuture.get.aclBindingDeleteResults().asScala.head.exception.isPresent) - TestUtils.waitAndVerifyAcls(prevAcls -- Set(ace), authorizer, clusterResourcePattern) - } - - private def clusterAcl(permissionType: AclPermissionType, operation: AclOperation): AccessControlEntry = { - new AccessControlEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*").toString, - WILDCARD_HOST, operation, permissionType) - } - } } diff --git a/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala index e1cf49b5af7bf..65906032ca4c1 100644 --- a/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala @@ -14,16 +14,21 @@ package kafka.api import java.util import java.util.concurrent._ +import java.util.Properties + import com.yammer.metrics.core.Gauge import kafka.security.authorizer.AclAuthorizer import kafka.utils.TestUtils -import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateAclsResult} +import org.apache.kafka.clients.admin.CreateAclsResult import org.apache.kafka.common.acl._ +import org.apache.kafka.common.config.SslConfigs +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType} -import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} +import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, SecurityProtocol, SslAuthenticationContext} +import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder import org.apache.kafka.server.authorizer._ -import org.apache.kafka.server.config.ZkConfigs +import org.apache.kafka.common.network.Mode import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotNull, assertTrue} import org.junit.jupiter.api.{AfterEach, Test} @@ -35,6 +40,9 @@ object SslAdminIntegrationTest { @volatile var semaphore: Option[Semaphore] = None @volatile var executor: Option[ExecutorService] = None @volatile var lastUpdateRequestContext: Option[AuthorizableRequestContext] = None + val superuserCn = "super-user" + val serverUser = "server" + class TestableAclAuthorizer extends AclAuthorizer { override def createAcls(requestContext: AuthorizableRequestContext, aclBindings: util.List[AclBinding]): util.List[_ <: CompletionStage[AclCreateResult]] = { @@ -74,16 +82,31 @@ object SslAdminIntegrationTest { futures.asJava } } + + class TestPrincipalBuilder extends DefaultKafkaPrincipalBuilder(null, null) { + private val Pattern = "O=A (.*?),CN=(.*?)".r + + // Use fields from DN as server principal to grant authorisation for servers and super admin client + override def build(context: AuthenticationContext): KafkaPrincipal = { + val peerPrincipal = context.asInstanceOf[SslAuthenticationContext].session.getPeerPrincipal.getName + peerPrincipal match { + case Pattern(name, cn) => + val principal = if ((name == "server") || (cn == superuserCn)) "server" else KafkaPrincipal.ANONYMOUS.getName + new KafkaPrincipal(KafkaPrincipal.USER_TYPE, principal) + case _ => + KafkaPrincipal.ANONYMOUS + } + } + } } class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest { - override val authorizationAdmin = new AclAuthorizationAdmin(classOf[SslAdminIntegrationTest.TestableAclAuthorizer], classOf[AclAuthorizer]) - - this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true") + override val aclAuthorizerClassName: String = classOf[SslAdminIntegrationTest.TestableAclAuthorizer].getName + this.serverConfig.setProperty(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required") + this.serverConfig.setProperty(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[SslAdminIntegrationTest.TestPrincipalBuilder].getName) override protected def securityProtocol = SecurityProtocol.SSL - override protected lazy val trustStoreFile = Some(TestUtils.tempFile("truststore", ".jks")) - private val adminClients = mutable.Buffer.empty[Admin] + override def kafkaPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, SslAdminIntegrationTest.serverUser) override def setUpSasl(): Unit = { SslAdminIntegrationTest.semaphore = None @@ -100,7 +123,6 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest { SslAdminIntegrationTest.semaphore = None semaphore.foreach(s => s.release(s.getQueueLength)) - adminClients.foreach(_.close()) super.tearDown() } @@ -206,7 +228,7 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest { val testSemaphore = new Semaphore(0) SslAdminIntegrationTest.semaphore = Some(testSemaphore) - client = Admin.create(createConfig) + client = createAdminClient val results = client.createAcls(List(acl2, acl3).asJava).values assertEquals(Set(acl2, acl3), results.keySet().asScala) assertFalse(results.values.asScala.exists(_.isDone)) @@ -228,14 +250,6 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest { validateRequestContext(SslAdminIntegrationTest.lastUpdateRequestContext.get, ApiKeys.DELETE_ACLS) } - private def createAdminClient: Admin = { - val config = createConfig - config.put(AdminClientConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, "40000") - val client = Admin.create(config) - adminClients += client - client - } - private def blockedRequestThreads: List[Thread] = { val requestThreads = Thread.getAllStackTraces.keySet.asScala .filter(_.getName.contains("data-plane-kafka-request-handler")) @@ -258,4 +272,12 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest { assertTrue(metrics.nonEmpty, s"Unable to find metric $name: allMetrics: ${allMetrics.keySet.map(_.getMBeanName)}") metrics.map(_.asInstanceOf[Gauge[Int]].value).sum } + + // Override the CN to create a principal based on it + override def superuserSecurityProps(certAlias: String): Properties = { + val props = TestUtils.securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, certAlias, SslAdminIntegrationTest.superuserCn, + clientSaslProperties) + props.remove(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG) + props + } } diff --git a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala index fa13715b87c20..3cb508958a482 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.server.util.ShutdownableThread -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -46,10 +46,10 @@ class TransactionsBounceTest extends IntegrationTestHarness { val overridingProps = new Properties() overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString) - overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) + overridingProps.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, serverMessageMaxBytes.toString) // Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic) // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long - overridingProps.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + overridingProps.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) overridingProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) overridingProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString) diff --git a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala index ee39a764f2ece..a31385224fd08 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala @@ -29,7 +29,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{InvalidPidMappingException, TransactionalIdNotFoundException} import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -208,7 +208,7 @@ class TransactionsExpirationTest extends KafkaServerTestHarness { serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 3.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, 2.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, 2.toString) - serverProps.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + serverProps.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) serverProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) serverProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) serverProps.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index 993e31dc5b0fd..09462d361ff01 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -17,7 +17,6 @@ package kafka.api -import kafka.server.KafkaConfig import kafka.utils.TestUtils import kafka.utils.TestUtils.{consumeRecords, waitUntilTrue} import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerGroupMetadata, ConsumerRecord, OffsetAndMetadata} @@ -26,7 +25,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{InvalidProducerEpochException, ProducerFencedException, TimeoutException} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -67,17 +66,11 @@ class TransactionsTest extends IntegrationTestHarness { props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 3.toString) props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, 2.toString) props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, 2.toString) - props.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + props.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) props.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) props.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) props.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") props.put(TransactionStateManagerConfigs.TRANSACTIONS_ABORT_TIMED_OUT_TRANSACTION_CLEANUP_INTERVAL_MS_CONFIG, "200") - - // The new group coordinator does not support verifying transactions yet. - if (isNewGroupCoordinatorEnabled()) { - props.put(TransactionLogConfigs.TRANSACTION_PARTITION_VERIFICATION_ENABLE_CONFIG, "false") - } - props } diff --git a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala index 4884fc2dc7b56..ae73cde5e848f 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.clients.consumer.Consumer import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -110,7 +110,7 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 1.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, 1.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, 1.toString) - serverProps.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + serverProps.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) serverProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) serverProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) serverProps.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") diff --git a/core/src/test/scala/integration/kafka/api/UserClientIdQuotaTest.scala b/core/src/test/scala/integration/kafka/api/UserClientIdQuotaTest.scala index a29a108d77bff..eac85dc2afda2 100644 --- a/core/src/test/scala/integration/kafka/api/UserClientIdQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/UserClientIdQuotaTest.scala @@ -16,9 +16,9 @@ package kafka.api import kafka.server._ import kafka.utils.TestUtils +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.utils.Sanitizer -import org.apache.kafka.server.config.KafkaSecurityConfigs import org.junit.jupiter.api.{BeforeEach, TestInfo} class UserClientIdQuotaTest extends BaseQuotaTest { @@ -31,7 +31,7 @@ class UserClientIdQuotaTest extends BaseQuotaTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - this.serverConfig.setProperty(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required") + this.serverConfig.setProperty(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required") super.setUp(testInfo) quotaTestClients.alterClientQuotas( quotaTestClients.clientQuotaAlteration( diff --git a/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala b/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala index c42fbd6330e7b..8fa1d2dca6df6 100644 --- a/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala +++ b/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala @@ -17,13 +17,13 @@ */ package kafka.network -import kafka.server.{BaseRequestTest, KafkaConfig} +import kafka.server.BaseRequestTest import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.network.SocketServerConfigs -import org.apache.kafka.server.config.Defaults +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -43,8 +43,8 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { properties.put(SocketServerConfigs.LISTENERS_CONFIG, s"$internal://localhost:0, $external://localhost:0") properties.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"$internal:PLAINTEXT, $external:PLAINTEXT") - properties.put(s"listener.name.${internal.toLowerCase}.${KafkaConfig.NumNetworkThreadsProp}", "2") - properties.put(KafkaConfig.NumNetworkThreadsProp, Defaults.NUM_NETWORK_THREADS.toString) + properties.put(s"listener.name.${internal.toLowerCase}.${ServerConfigs.NUM_NETWORK_THREADS_CONFIG}", "2") + properties.put(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, ServerConfigs.NUM_NETWORK_THREADS_DEFAULT.toString) } @BeforeEach @@ -53,7 +53,7 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { admin = TestUtils.createAdminClient(brokers, ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)) assertEquals(2, getNumNetworkThreads(internal)) TestUtils.createTopicWithAdmin(admin, "test", brokers, controllerServers) - assertEquals(Defaults.NUM_NETWORK_THREADS, getNumNetworkThreads(external)) + assertEquals(ServerConfigs.NUM_NETWORK_THREADS_DEFAULT, getNumNetworkThreads(external)) } @AfterEach override def tearDown(): Unit = { @@ -71,10 +71,10 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { @ValueSource(strings = Array("zk", "kraft")) def testDynamicNumNetworkThreads(quorum: String): Unit = { // Increase the base network thread count - val newBaseNetworkThreadsCount = Defaults.NUM_NETWORK_THREADS + 1 + val newBaseNetworkThreadsCount = ServerConfigs.NUM_NETWORK_THREADS_DEFAULT + 1 var props = new Properties - props.put(KafkaConfig.NumNetworkThreadsProp, newBaseNetworkThreadsCount.toString) - reconfigureServers(props, (KafkaConfig.NumNetworkThreadsProp, newBaseNetworkThreadsCount.toString)) + props.put(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, newBaseNetworkThreadsCount.toString) + reconfigureServers(props, (ServerConfigs.NUM_NETWORK_THREADS_CONFIG, newBaseNetworkThreadsCount.toString)) // Only the external listener is changed assertEquals(2, getNumNetworkThreads(internal)) @@ -83,8 +83,8 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { // Increase the network thread count for internal val newInternalNetworkThreadsCount = 3 props = new Properties - props.put(s"listener.name.${internal.toLowerCase}.${KafkaConfig.NumNetworkThreadsProp}", newInternalNetworkThreadsCount.toString) - reconfigureServers(props, (s"listener.name.${internal.toLowerCase}.${KafkaConfig.NumNetworkThreadsProp}", newInternalNetworkThreadsCount.toString)) + props.put(s"listener.name.${internal.toLowerCase}.${ServerConfigs.NUM_NETWORK_THREADS_CONFIG}", newInternalNetworkThreadsCount.toString) + reconfigureServers(props, (s"listener.name.${internal.toLowerCase}.${ServerConfigs.NUM_NETWORK_THREADS_CONFIG}", newInternalNetworkThreadsCount.toString)) // The internal listener is changed assertEquals(newInternalNetworkThreadsCount, getNumNetworkThreads(internal)) diff --git a/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala b/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala index f26f7079d4bfe..ccf4b84b696fe 100644 --- a/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala +++ b/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala @@ -28,6 +28,8 @@ import org.apache.kafka.common.requests.FetchRequest import org.apache.kafka.storage.internals.log.{FetchDataInfo, FetchIsolation, FetchParams, FetchPartitionData, LogOffsetMetadata, LogOffsetSnapshot} import org.junit.jupiter.api.Test import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.Mockito.{mock, when} @@ -46,7 +48,7 @@ class DelayedFetchTest { val fetchStatus = FetchPartitionStatus( startOffsetMetadata = new LogOffsetMetadata(fetchOffset), - fetchInfo = new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) + fetchInfo = new FetchRequest.PartitionData(topicIdPartition.topicId(), fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500) var fetchResultOpt: Option[FetchPartitionData] = None @@ -92,7 +94,7 @@ class DelayedFetchTest { val fetchStatus = FetchPartitionStatus( startOffsetMetadata = new LogOffsetMetadata(fetchOffset), - fetchInfo = new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) + fetchInfo = new FetchRequest.PartitionData(topicIdPartition.topicId(), fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500) var fetchResultOpt: Option[FetchPartitionData] = None @@ -116,6 +118,9 @@ class DelayedFetchTest { assertTrue(delayedFetch.tryComplete()) assertTrue(delayedFetch.isCompleted) assertTrue(fetchResultOpt.isDefined) + + val fetchResult = fetchResultOpt.get + assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, fetchResult.error) } @Test @@ -164,18 +169,71 @@ class DelayedFetchTest { assertTrue(delayedFetch.tryComplete()) assertTrue(delayedFetch.isCompleted) assertTrue(fetchResultOpt.isDefined) + + val fetchResult = fetchResultOpt.get + assertEquals(Errors.NONE, fetchResult.error) + } + + @ParameterizedTest(name = "testDelayedFetchWithMessageOnlyHighWatermark endOffset={0}") + @ValueSource(longs = Array(0, 500)) + def testDelayedFetchWithMessageOnlyHighWatermark(endOffset: Long): Unit = { + val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "topic") + val fetchOffset = 450L + val logStartOffset = 5L + val currentLeaderEpoch = Optional.of[Integer](10) + val replicaId = 1 + + val fetchStatus = FetchPartitionStatus( + startOffsetMetadata = new LogOffsetMetadata(fetchOffset), + fetchInfo = new FetchRequest.PartitionData(topicIdPartition.topicId, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) + val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500) + + var fetchResultOpt: Option[FetchPartitionData] = None + def callback(responses: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = { + fetchResultOpt = Some(responses.head._2) + } + + val delayedFetch = new DelayedFetch( + params = fetchParams, + fetchPartitionStatus = Seq(topicIdPartition -> fetchStatus), + replicaManager = replicaManager, + quota = replicaQuota, + responseCallback = callback + ) + + val partition: Partition = mock(classOf[Partition]) + when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition)).thenReturn(partition) + // Note that the high-watermark does not contain the complete metadata + val endOffsetMetadata = new LogOffsetMetadata(endOffset, -1L, -1) + when(partition.fetchOffsetSnapshot( + currentLeaderEpoch, + fetchOnlyFromLeader = true)) + .thenReturn(new LogOffsetSnapshot(0L, endOffsetMetadata, endOffsetMetadata, endOffsetMetadata)) + when(replicaManager.isAddingReplica(any(), anyInt())).thenReturn(false) + expectReadFromReplica(fetchParams, topicIdPartition, fetchStatus.fetchInfo, Errors.NONE) + + // 1. When `endOffset` is 0, it refers to the truncation case + // 2. When `endOffset` is 500, we won't complete because it doesn't contain offset metadata + val expected = endOffset == 0 + assertEquals(expected, delayedFetch.tryComplete()) + assertEquals(expected, delayedFetch.isCompleted) + assertEquals(expected, fetchResultOpt.isDefined) + if (fetchResultOpt.isDefined) { + assertEquals(Errors.NONE, fetchResultOpt.get.error) + } } private def buildFollowerFetchParams( replicaId: Int, - maxWaitMs: Int + maxWaitMs: Int, + minBytes: Int = 1, ): FetchParams = { new FetchParams( ApiKeys.FETCH.latestVersion, replicaId, 1, maxWaitMs, - 1, + minBytes, maxBytes, FetchIsolation.LOG_END, Optional.empty() diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 379b454bc84f5..8508607eefde5 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -44,8 +44,9 @@ import org.apache.kafka.clients.admin._ import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecord, ConsumerRecords, KafkaConsumer} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.{ClusterResource, ClusterResourceListener, Reconfigurable, TopicPartition, TopicPartitionInfo} -import org.apache.kafka.common.config.{ConfigException, ConfigResource} +import org.apache.kafka.common.config.{ConfigException, ConfigResource, SaslConfigs} import org.apache.kafka.common.config.SslConfigs._ +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.config.provider.FileConfigProvider import org.apache.kafka.common.errors.{AuthenticationException, InvalidRequestException} @@ -62,7 +63,7 @@ import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.{PasswordEncoder, PasswordEncoderConfigs} -import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{ConfigType, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.metrics.{KafkaYammerMetrics, MetricConfigs} import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.util.ShutdownableThread @@ -132,9 +133,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props.put(SocketServerConfigs.LISTENERS_CONFIG, s"$SecureInternal://localhost:0, $SecureExternal://localhost:0") props.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"PLAINTEXT:PLAINTEXT, $SecureInternal:SSL, $SecureExternal:SASL_SSL, CONTROLLER:$controllerListenerSecurityProtocol") props.put(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, SecureInternal) - props.put(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "requested") - props.put(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, "PLAIN") - props.put(KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, kafkaServerSaslMechanisms.mkString(",")) + props.put(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "requested") + props.put(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, "PLAIN") + props.put(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, kafkaServerSaslMechanisms.mkString(",")) props.put(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, "2000") // low value to test log rolling on config update props.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "2") // greater than one to test reducing threads props.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, "dynamic-config-secret") @@ -542,7 +543,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "20000000") props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, "0.8") props.put(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, "300000") - props.put(KafkaConfig.MessageMaxBytesProp, "40000") + props.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "40000") props.put(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, "50000000") props.put(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, "6000") @@ -634,7 +635,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props.put(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG, "60000") props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "10000000") props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, TimeUnit.DAYS.toMillis(1).toString) - props.put(KafkaConfig.MessageMaxBytesProp, "100000") + props.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "100000") props.put(ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_CONFIG, "10000") props.put(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, TimeUnit.DAYS.toMillis(1).toString) props.put(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP, "60000") @@ -643,7 +644,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props.put(ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG, "delete") props.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "false") props.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "2") - props.put(KafkaConfig.CompressionTypeProp, "gzip") + props.put(ServerConfigs.COMPRESSION_TYPE_CONFIG, "gzip") props.put(ServerLogConfigs.LOG_PRE_ALLOCATE_CONFIG, true.toString) props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.LOG_APPEND_TIME.toString) props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, "1000") @@ -869,15 +870,15 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } val config = servers.head.config - verifyThreadPoolResize(KafkaConfig.NumIoThreadsProp, config.numIoThreads, + verifyThreadPoolResize(ServerConfigs.NUM_IO_THREADS_CONFIG, config.numIoThreads, requestHandlerPrefix, mayReceiveDuplicates = false) verifyThreadPoolResize(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, config.numReplicaFetchers, fetcherThreadPrefix, mayReceiveDuplicates = false) - verifyThreadPoolResize(KafkaConfig.BackgroundThreadsProp, config.backgroundThreads, + verifyThreadPoolResize(ServerConfigs.BACKGROUND_THREADS_CONFIG, config.backgroundThreads, "kafka-scheduler-", mayReceiveDuplicates = false) verifyThreadPoolResize(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, config.numRecoveryThreadsPerDataDir, "", mayReceiveDuplicates = false) - verifyThreadPoolResize(KafkaConfig.NumNetworkThreadsProp, config.numNetworkThreads, + verifyThreadPoolResize(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, config.numNetworkThreads, networkThreadPrefix, mayReceiveDuplicates = true) verifyThreads("data-plane-kafka-socket-acceptor-", config.listeners.size) @@ -1743,12 +1744,12 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup private def addListenerPropsSasl(listener: String, mechanisms: Seq[String], props: Properties): Unit = { val listenerName = new ListenerName(listener) val prefix = listenerName.configPrefix - props.put(prefix + KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, mechanisms.mkString(",")) - props.put(prefix + KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG, "kafka") + props.put(prefix + BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, mechanisms.mkString(",")) + props.put(prefix + SaslConfigs.SASL_KERBEROS_SERVICE_NAME, "kafka") mechanisms.foreach { mechanism => val jaasSection = jaasSections(Seq(mechanism), None, KafkaSasl, "").head val jaasConfig = jaasSection.modules.head.toString - props.put(listenerName.saslMechanismConfigPrefix(mechanism) + KafkaSecurityConfigs.SASL_JAAS_CONFIG, jaasConfig) + props.put(listenerName.saslMechanismConfigPrefix(mechanism) + SaslConfigs.SASL_JAAS_CONFIG, jaasConfig) } } @@ -1965,7 +1966,7 @@ class TestMetricsReporter extends MetricsReporter with Reconfigurable with Close } override def configure(configs: util.Map[String, _]): Unit = { - configuredBrokers += configs.get(KafkaConfig.BrokerIdProp).toString.toInt + configuredBrokers += configs.get(ServerConfigs.BROKER_ID_CONFIG).toString.toInt configureCount += 1 pollingInterval = configs.get(PollingIntervalProp).toString.toInt } diff --git a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala index dde38e6d35f99..c507d46a8eb42 100644 --- a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala +++ b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala @@ -28,6 +28,7 @@ import kafka.utils.TestUtils import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.config.SaslConfigs +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.errors.SaslAuthenticationException import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.network._ @@ -36,7 +37,6 @@ import org.apache.kafka.common.security.auth.{Login, SecurityProtocol} import org.apache.kafka.common.security.kerberos.KerberosLogin import org.apache.kafka.common.utils.{LogContext, MockTime} import org.apache.kafka.network.SocketServerConfigs -import org.apache.kafka.server.config.KafkaSecurityConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -63,7 +63,7 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { override def setUp(testInfo: TestInfo): Unit = { TestableKerberosLogin.reset() startSasl(jaasSections(kafkaServerSaslMechanisms, Option(kafkaClientSaslMechanism), Both)) - serverConfig.put(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required") + serverConfig.put(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required") serverConfig.put(SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG, failedAuthenticationDelayMs.toString) super.setUp(testInfo) serverAddr = new InetSocketAddress("localhost", diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 67606c39e219e..8be2b994116f5 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -1072,6 +1072,49 @@ class KRaftClusterTest { } } + @Test + def testCreateClusterAndCreateTopicWithRemoteLogManagerInstantiation(): Unit = { + val cluster = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(1).build()) + .setConfigProp(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, true.toString) + .setConfigProp(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, + "org.apache.kafka.server.log.remote.storage.NoOpRemoteLogMetadataManager") + .setConfigProp(RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, + "org.apache.kafka.server.log.remote.storage.NoOpRemoteStorageManager") + .build() + try { + cluster.format() + cluster.startup() + cluster.waitForReadyBrokers() + TestUtils.waitUntilTrue(() => cluster.brokers().get(0).brokerState == BrokerState.RUNNING, + "Broker never made it to RUNNING state.") + TestUtils.waitUntilTrue(() => cluster.raftManagers().get(0).client.leaderAndEpoch().leaderId.isPresent, + "RaftManager was not initialized.") + + val admin = Admin.create(cluster.clientProperties()) + try { + // Create a test topic + val newTopic = Collections.singletonList(new NewTopic("test-topic", 1, 1.toShort)) + val createTopicResult = admin.createTopics(newTopic) + createTopicResult.all().get() + waitForTopicListing(admin, Seq("test-topic"), Seq()) + + // Delete topic + val deleteResult = admin.deleteTopics(Collections.singletonList("test-topic")) + deleteResult.all().get() + + // List again + waitForTopicListing(admin, Seq(), Seq("test-topic")) + } finally { + admin.close() + } + } finally { + cluster.close() + } + } + @Test def testSnapshotCount(): Unit = { val cluster = new KafkaClusterTestKit.Builder( @@ -1541,6 +1584,44 @@ class KRaftClusterTest { cluster.close() } } + + @Test + def testControllerFailover(): Unit = { + val cluster = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(5).build()).build() + try { + cluster.format() + cluster.startup() + cluster.waitForReadyBrokers() + TestUtils.waitUntilTrue(() => cluster.brokers().get(0).brokerState == BrokerState.RUNNING, + "Broker never made it to RUNNING state.") + TestUtils.waitUntilTrue(() => cluster.raftManagers().get(0).client.leaderAndEpoch().leaderId.isPresent, + "RaftManager was not initialized.") + + val admin = Admin.create(cluster.clientProperties()) + try { + // Create a test topic + admin.createTopics(Collections.singletonList( + new NewTopic("test-topic", 1, 1.toShort))).all().get() + waitForTopicListing(admin, Seq("test-topic"), Seq()) + + // Shut down active controller + val active = cluster.waitForActiveController() + cluster.raftManagers().get(active.asInstanceOf[QuorumController].nodeId()).shutdown() + + // Create a test topic on the new active controller + admin.createTopics(Collections.singletonList( + new NewTopic("test-topic2", 1, 1.toShort))).all().get() + waitForTopicListing(admin, Seq("test-topic2"), Seq()) + } finally { + admin.close() + } + } finally { + cluster.close() + } + } } class BadAuthorizer extends Authorizer { diff --git a/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala b/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala index 56a5cda902cf0..fed241f5d7b09 100644 --- a/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala +++ b/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala @@ -26,10 +26,11 @@ import kafka.utils.{JaasTestUtils, TestUtils} import kafka.utils.Implicits._ import org.apache.kafka.clients.consumer.Consumer import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} -import org.apache.kafka.common.config.SslConfigs +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs +import org.apache.kafka.common.config.{SaslConfigs, SslConfigs} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.network.{ListenerName, Mode} -import org.apache.kafka.server.config.{KafkaSecurityConfigs, ReplicationConfigs, ZkConfigs} +import org.apache.kafka.server.config.{ReplicationConfigs, ZkConfigs} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.network.SocketServerConfigs import org.junit.jupiter.api.Assertions.assertEquals @@ -82,12 +83,12 @@ abstract class MultipleListenersWithSameSecurityProtocolBaseTest extends QuorumT s"$External:PLAINTEXT, $SecureExternal:SASL_SSL") props.put(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, Internal) props.put(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true") - props.put(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, kafkaClientSaslMechanism) - props.put(s"${new ListenerName(SecureInternal).configPrefix}${KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG}", + props.put(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, kafkaClientSaslMechanism) + props.put(s"${new ListenerName(SecureInternal).configPrefix}${BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG}", kafkaServerSaslMechanisms(SecureInternal).mkString(",")) - props.put(s"${new ListenerName(SecureExternal).configPrefix}${KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG}", + props.put(s"${new ListenerName(SecureExternal).configPrefix}${BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG}", kafkaServerSaslMechanisms(SecureExternal).mkString(",")) - props.put(KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG, "kafka") + props.put(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, "kafka") props ++= dynamicJaasSections props ++= TestUtils.sslConfigs(Mode.SERVER, clientCert = false, Some(trustStoreFile), s"server$brokerId") @@ -176,7 +177,7 @@ abstract class MultipleListenersWithSameSecurityProtocolBaseTest extends QuorumT val listenerName = new ListenerName(listener) val prefix = listenerName.saslMechanismConfigPrefix(mechanism) val jaasConfig = jaasSection.modules.head.toString - props.put(s"${prefix}${KafkaSecurityConfigs.SASL_JAAS_CONFIG}", jaasConfig) + props.put(s"${prefix}${SaslConfigs.SASL_JAAS_CONFIG}", jaasConfig) } case class ClientMetadata(listenerName: ListenerName, saslMechanism: String, topic: String) { diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index abc92c8c9267f..9f787a1b16881 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -37,8 +37,8 @@ import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationF import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} -import org.apache.kafka.server.config.KRaftConfigs +import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion} +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs} import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler} import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.{WatchedEvent, Watcher, ZooKeeper} @@ -324,7 +324,7 @@ abstract class QuorumTestHarness extends Logging { props.putAll(overridingProps) props.setProperty(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG, TimeUnit.MINUTES.toMillis(10).toString) props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller") - props.setProperty(KafkaConfig.UnstableMetadataVersionsEnableProp, "true") + props.setProperty(ServerConfigs.UNSTABLE_FEATURE_VERSIONS_ENABLE_CONFIG, "true") if (props.getProperty(KRaftConfigs.NODE_ID_CONFIG) == null) { props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1000") } @@ -342,6 +342,15 @@ abstract class QuorumTestHarness extends Logging { setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(metadataVersion.featureLevel()), 0.toShort)) + if (isNewGroupCoordinatorEnabled()) { + metadataRecords.add(new ApiMessageAndVersion( + new FeatureLevelRecord() + .setName(Features.GROUP_VERSION.featureName) + .setFeatureLevel(Features.GROUP_VERSION.latestTesting), + 0.toShort + )) + } + optionalMetadataRecords.foreach { metadataArguments => for (record <- metadataArguments) metadataRecords.add(record) } diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala index 5fd32d8f4bacb..c0b6d916ec228 100644 --- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala @@ -72,7 +72,8 @@ object ZkMigrationIntegrationTest { MetadataVersion.IBP_3_7_IV1, MetadataVersion.IBP_3_7_IV2, MetadataVersion.IBP_3_7_IV4, - MetadataVersion.IBP_3_8_IV0 + MetadataVersion.IBP_3_8_IV0, + MetadataVersion.IBP_4_0_IVO ).map { mv => val serverProperties = new util.HashMap[String, String]() serverProperties.put("inter.broker.listener.name", "EXTERNAL") diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 5bf8f32884572..4f2daf3bfa6f3 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -184,6 +184,24 @@ final class KafkaMetadataLogTest { ) } + @Test + def testHighWatermarkOffsetMetadata(): Unit = { + val numberOfRecords = 10 + val epoch = 1 + val log = buildMetadataLog(tempDir, mockTime) + + append(log, numberOfRecords, epoch) + log.updateHighWatermark(new LogOffsetMetadata(numberOfRecords)) + + val highWatermarkMetadata = log.highWatermark + assertEquals(numberOfRecords, highWatermarkMetadata.offset) + assertTrue(highWatermarkMetadata.metadata.isPresent) + + val segmentPosition = highWatermarkMetadata.metadata.get().asInstanceOf[SegmentPosition] + assertEquals(0, segmentPosition.baseOffset) + assertTrue(segmentPosition.relativePosition > 0) + } + @Test def testCreateSnapshotBeforeLogStartOffset(): Unit = { val numberOfRecords = 10 diff --git a/core/src/test/scala/kafka/server/KafkaRequestHandlerTest.scala b/core/src/test/scala/kafka/server/KafkaRequestHandlerTest.scala index ba584c9a6e738..7be9b47c12465 100644 --- a/core/src/test/scala/kafka/server/KafkaRequestHandlerTest.scala +++ b/core/src/test/scala/kafka/server/KafkaRequestHandlerTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.{RequestContext, RequestHeader} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.utils.{BufferSupplier, MockTime, Time} -import org.apache.kafka.server.log.remote.storage.{RemoteLogManagerConfig, RemoteStorageMetrics} +import org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.Test import org.junit.jupiter.params.ParameterizedTest @@ -40,13 +40,11 @@ import java.util.concurrent.CompletableFuture import java.util.concurrent.atomic.AtomicInteger class KafkaRequestHandlerTest { - val props = kafka.utils.TestUtils.createDummyBrokerConfig() - props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, true.toString) - val brokerTopicStats = new BrokerTopicStats(java.util.Optional.of(KafkaConfig.fromProps(props))) + val brokerTopicStats = new BrokerTopicStats(true) val topic = "topic" val topic2 = "topic2" - val brokerTopicMetrics = brokerTopicStats.topicStats(topic) - val allTopicMetrics = brokerTopicStats.allTopicsStats + val brokerTopicMetrics: BrokerTopicMetrics = brokerTopicStats.topicStats(topic) + val allTopicMetrics: BrokerTopicMetrics = brokerTopicStats.allTopicsStats @Test def testCallbackTiming(): Unit = { @@ -65,7 +63,7 @@ class KafkaRequestHandlerTest { time.sleep(2) // Prepare the callback. val callback = KafkaRequestHandler.wrapAsyncCallback( - (reqLocal: RequestLocal, ms: Int) => { + (_: RequestLocal, ms: Int) => { time.sleep(ms) handler.stop() }, @@ -104,7 +102,7 @@ class KafkaRequestHandlerTest { handledCount = handledCount + 1 // Prepare the callback. val callback = KafkaRequestHandler.wrapAsyncCallback( - (reqLocal: RequestLocal, ms: Int) => { + (_: RequestLocal, _: Int) => { handler.stop() }, RequestLocal.NoCaching) @@ -140,7 +138,7 @@ class KafkaRequestHandlerTest { when(apiHandler.handle(ArgumentMatchers.eq(request), any())).thenAnswer { _ => // Prepare the callback. val callback = KafkaRequestHandler.wrapAsyncCallback( - (reqLocal: RequestLocal, ms: Int) => { + (reqLocal: RequestLocal, _: Int) => { reqLocal.bufferSupplier.close() handledCount = handledCount + 1 handler.stop() @@ -175,7 +173,7 @@ class KafkaRequestHandlerTest { when(apiHandler.handle(ArgumentMatchers.eq(request), any())).thenAnswer { _ => // Prepare the callback. val callback = KafkaRequestHandler.wrapAsyncCallback( - (reqLocal: RequestLocal, ms: Int) => { + (reqLocal: RequestLocal, _: Int) => { reqLocal.bufferSupplier.close() handledCount = handledCount + 1 handler.stop() @@ -195,9 +193,7 @@ class KafkaRequestHandlerTest { @ParameterizedTest @ValueSource(booleans = Array(true, false)) def testTopicStats(systemRemoteStorageEnabled: Boolean): Unit = { - val props = kafka.utils.TestUtils.createDummyBrokerConfig() - props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, systemRemoteStorageEnabled.toString) - val brokerTopicStats = new BrokerTopicStats(java.util.Optional.of(KafkaConfig.fromProps(props))) + val brokerTopicStats = new BrokerTopicStats(systemRemoteStorageEnabled) val brokerTopicMetrics = brokerTopicStats.topicStats(topic) val gaugeMetrics = Set( RemoteStorageMetrics.REMOTE_LOG_SIZE_COMPUTATION_TIME_METRIC.getName, @@ -243,17 +239,13 @@ class KafkaRequestHandlerTest { def setupBrokerTopicMetrics(systemRemoteStorageEnabled: Boolean = true): BrokerTopicMetrics = { val topic = "topic" - val props = kafka.utils.TestUtils.createDummyBrokerConfig() - props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, systemRemoteStorageEnabled.toString) - new BrokerTopicMetrics(Option.apply(topic), java.util.Optional.of(KafkaConfig.fromProps(props))) + new BrokerTopicMetrics(Option.apply(topic), systemRemoteStorageEnabled) } @ParameterizedTest @ValueSource(booleans = Array(true, false)) def testSingularCopyLagBytesMetric(systemRemoteStorageEnabled: Boolean): Unit = { - val props = kafka.utils.TestUtils.createDummyBrokerConfig() - props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, systemRemoteStorageEnabled.toString) - val brokerTopicStats = new BrokerTopicStats(java.util.Optional.of(KafkaConfig.fromProps(props))) + val brokerTopicStats = new BrokerTopicStats(systemRemoteStorageEnabled) val brokerTopicMetrics = brokerTopicStats.topicStats(topic) if (systemRemoteStorageEnabled) { @@ -593,9 +585,7 @@ class KafkaRequestHandlerTest { @ParameterizedTest @ValueSource(booleans = Array(true, false)) def testSingularLogSizeBytesMetric(systemRemoteStorageEnabled: Boolean): Unit = { - val props = kafka.utils.TestUtils.createDummyBrokerConfig() - props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, systemRemoteStorageEnabled.toString) - val brokerTopicStats = new BrokerTopicStats(java.util.Optional.of(KafkaConfig.fromProps(props))) + val brokerTopicStats = new BrokerTopicStats(systemRemoteStorageEnabled) val brokerTopicMetrics = brokerTopicStats.topicStats(topic) if (systemRemoteStorageEnabled) { brokerTopicStats.recordRemoteLogSizeBytes(topic, 0, 100) diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index 73359c5a7d6d1..457326cd19adf 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -22,13 +22,13 @@ import java.util.Properties import kafka.server.KafkaConfig import kafka.utils.{Exit, TestUtils} import kafka.utils.TestUtils.assertBadConfigContainingMessage +import org.apache.kafka.common.config.SslConfigs import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.internals.FatalExitError import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.QuorumConfig - -import org.apache.kafka.server.config.{KRaftConfigs, KafkaSecurityConfigs, ZkConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, ZkConfigs} import org.apache.kafka.server.config.ReplicationConfigs import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.Assertions._ @@ -198,19 +198,19 @@ class KafkaConfigTest { "--override", "ssl.keystore.certificate.chain=certificate_chain", "--override", "ssl.keystore.key=private_key", "--override", "ssl.truststore.certificates=truststore_certificates"))) - assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG).toString) - assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).toString) - assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).toString) - assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_KEY_CONFIG).toString) - assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG).toString) - assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG).toString) - - assertEquals("key_password", config.getPassword(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG).value) - assertEquals("keystore_password", config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).value) - assertEquals("truststore_password", config.getPassword(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value) - assertEquals("private_key", config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_KEY_CONFIG).value) - assertEquals("certificate_chain", config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG).value) - assertEquals("truststore_certificates", config.getPassword(KafkaSecurityConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG).value) + assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_KEY_PASSWORD_CONFIG).toString) + assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).toString) + assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).toString) + assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_KEYSTORE_KEY_CONFIG).toString) + assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG).toString) + assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG).toString) + + assertEquals("key_password", config.getPassword(SslConfigs.SSL_KEY_PASSWORD_CONFIG).value) + assertEquals("keystore_password", config.getPassword(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).value) + assertEquals("truststore_password", config.getPassword(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value) + assertEquals("private_key", config.getPassword(SslConfigs.SSL_KEYSTORE_KEY_CONFIG).value) + assertEquals("certificate_chain", config.getPassword(SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG).value) + assertEquals("truststore_certificates", config.getPassword(SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG).value) } @Test @@ -221,13 +221,13 @@ class KafkaConfigTest { "--override", "ssl.keystore.password=" + password, "--override", "ssl.key.password=" + password, "--override", "ssl.truststore.password=" + password))) - assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG).toString) - assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).toString) - assertEquals(Password.HIDDEN, config.getPassword(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).toString) + assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_KEY_PASSWORD_CONFIG).toString) + assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).toString) + assertEquals(Password.HIDDEN, config.getPassword(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).toString) - assertEquals(password, config.getPassword(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).value) - assertEquals(password, config.getPassword(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG).value) - assertEquals(password, config.getPassword(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value) + assertEquals(password, config.getPassword(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG).value) + assertEquals(password, config.getPassword(SslConfigs.SSL_KEY_PASSWORD_CONFIG).value) + assertEquals(password, config.getPassword(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).value) } private val booleanPropValueToSet = true @@ -346,7 +346,7 @@ class KafkaConfigTest { def testConnectionsMaxReauthMsDefault(): Unit = { val propertiesFile = prepareDefaultConfig() val config = KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile))) - assertEquals(0L, config.valuesWithPrefixOverride("sasl_ssl.oauthbearer.").get(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS).asInstanceOf[Long]) + assertEquals(0L, config.valuesWithPrefixOverride("sasl_ssl.oauthbearer.").get(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG).asInstanceOf[Long]) } @Test @@ -354,7 +354,7 @@ class KafkaConfigTest { val propertiesFile = prepareDefaultConfig() val expected = 3600000 val config = KafkaConfig.fromProps(Kafka.getPropsFromArgs(Array(propertiesFile, "--override", s"sasl_ssl.oauthbearer.connections.max.reauth.ms=$expected"))) - assertEquals(expected, config.valuesWithPrefixOverride("sasl_ssl.oauthbearer.").get(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS).asInstanceOf[Long]) + assertEquals(expected, config.valuesWithPrefixOverride("sasl_ssl.oauthbearer.").get(BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG).asInstanceOf[Long]) } private def testZkConfig[T, U](kafkaPropName: String, diff --git a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala index 3327dc4073826..dbd84088b14f2 100644 --- a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala @@ -31,6 +31,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.utils.{AppInfoParser, SecurityUtils} import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.authorizer.Authorizer +import org.apache.kafka.server.config.ServerConfigs import org.apache.log4j.Level import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -110,7 +111,7 @@ class AclCommandTest extends QuorumTestHarness with Logging { super.setUp(testInfo) brokerProps = TestUtils.createBrokerConfig(0, zkConnect) - brokerProps.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) + brokerProps.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getName) brokerProps.put(AclAuthorizer.SuperUsersProp, "User:ANONYMOUS") zkArgs = Array("--authorizer-properties", "zookeeper.connect=" + zkConnect) diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index bb2e76e8b2d34..2ddbfa0c23c71 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -28,6 +28,7 @@ import org.apache.kafka.common.errors.{TopicDeletionDisabledException, UnknownTo import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.metadata.BrokerState +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.junit.jupiter.params.ParameterizedTest @@ -489,7 +490,7 @@ class DeleteTopicTest extends QuorumTestHarness { if (isKRaftTest()) { // Restart KRaft quorum with the updated config val overridingProps = new Properties() - overridingProps.put(KafkaConfig.DeleteTopicEnableProp, false.toString) + overridingProps.put(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, false.toString) if (implementation != null) implementation.shutdown() implementation = newKRaftQuorum(overridingProps) diff --git a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala index a363ef7f5f9bf..d8a932c2f67fe 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_10_2_IV0, IBP_0_9_0, IBP_1_0_IV0, IBP_2_2_IV0, IBP_2_4_IV0, IBP_2_4_IV1, IBP_2_6_IV0, IBP_2_8_IV1, IBP_3_2_IV0, IBP_3_4_IV0} -import org.apache.kafka.server.config.ZkConfigs +import org.apache.kafka.server.config.{ServerConfigs, ZkConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -895,7 +895,7 @@ class ControllerChannelManagerTest { private def createConfig(interBrokerVersion: MetadataVersion): KafkaConfig = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, controllerId.toString) + props.put(ServerConfigs.BROKER_ID_CONFIG, controllerId.toString) props.put(ZkConfigs.ZK_CONNECT_CONFIG, "zkConnect") TestUtils.setIbpAndMessageFormatVersions(props, interBrokerVersion) KafkaConfig.fromProps(props) 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 16459cbed2f79..f184b625bda87 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala @@ -1664,6 +1664,134 @@ class GroupMetadataManagerTest { assertEquals(0, TestUtils.totalMetricValue(metrics, "offset-commit-count")) } + @Test + def testOffsetMetadataTooLargePartialFailure(): Unit = { + val memberId = "" + val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "foo") + val validTopicIdPartition = new TopicIdPartition(topicIdPartition.topicId, 1, "foo") + val offset = 37 + val requireStable = true; + + groupMetadataManager.addOwnedPartition(groupPartitionId) + val group = new GroupMetadata(groupId, Empty, time) + groupMetadataManager.addGroup(group) + + val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + val offsets = immutable.Map( + topicIdPartition -> OffsetAndMetadata(offset, "s" * (offsetConfig.maxMetadataSize + 1) , time.milliseconds()), + validTopicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds()) + ) + + expectAppendMessage(Errors.NONE) + + var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None + def callback(errors: immutable.Map[TopicIdPartition, Errors]): Unit = { + commitErrors = Some(errors) + } + + assertEquals(0, TestUtils.totalMetricValue(metrics, "offset-commit-count")) + groupMetadataManager.storeOffsets(group, memberId, offsetTopicPartition, offsets, callback, verificationGuard = None) + assertTrue(group.hasOffsets) + + assertEquals(Some(Map( + topicIdPartition -> Errors.OFFSET_METADATA_TOO_LARGE, + validTopicIdPartition -> Errors.NONE) + ), commitErrors) + + val cachedOffsets = groupMetadataManager.getOffsets( + groupId, + requireStable, + Some(Seq(topicIdPartition.topicPartition, validTopicIdPartition.topicPartition)) + ) + + assertEquals( + Some(OffsetFetchResponse.INVALID_OFFSET), + cachedOffsets.get(topicIdPartition.topicPartition).map(_.offset) + ) + assertEquals( + Some(Errors.NONE), + cachedOffsets.get(topicIdPartition.topicPartition).map(_.error) + ) + assertEquals( + Some(offset), + cachedOffsets.get(validTopicIdPartition.topicPartition).map(_.offset) + ) + + assertEquals(1, TestUtils.totalMetricValue(metrics, "offset-commit-count")) + } + + @Test + def testTransactionalCommitOffsetWithOffsetMetadataTooLargePartialFailure(): Unit = { + val memberId = "" + val foo0 = new TopicIdPartition(Uuid.randomUuid(), 0, "foo") + val foo1 = new TopicIdPartition(Uuid.randomUuid(), 1, "foo") + val producerId = 232L + val producerEpoch = 0.toShort + + groupMetadataManager.addOwnedPartition(groupPartitionId) + + val group = new GroupMetadata(groupId, Empty, time) + groupMetadataManager.addGroup(group) + + val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + val offsets = immutable.Map( + foo0 -> OffsetAndMetadata(37, "", time.milliseconds()), + foo1 -> OffsetAndMetadata(38, "s" * (offsetConfig.maxMetadataSize + 1), time.milliseconds()) + ) + + val capturedResponseCallback: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = + ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) + var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None + + def callback(errors: immutable.Map[TopicIdPartition, Errors]): Unit = { + commitErrors = Some(errors) + } + + val verificationGuard = new VerificationGuard() + + groupMetadataManager.storeOffsets( + group, + memberId, + offsetTopicPartition, + offsets, + callback, + producerId, + producerEpoch, + verificationGuard = Some(verificationGuard) + ) + assertTrue(group.hasOffsets) + assertTrue(group.allOffsets.isEmpty) + + verify(replicaManager).appendRecords(anyLong(), + anyShort(), + any(), + any(), + any[Map[TopicPartition, MemoryRecords]], + capturedResponseCallback.capture(), + any[Option[ReentrantLock]], + any(), + any(), + any(), + ArgumentMatchers.eq(Map(offsetTopicPartition -> verificationGuard))) + verify(replicaManager).getMagic(any()) + capturedResponseCallback.getValue.apply(Map(groupTopicPartition -> + new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP, 0L))) + + assertEquals(Some(Map( + foo0 -> Errors.NONE, + foo1 -> Errors.OFFSET_METADATA_TOO_LARGE + )), commitErrors) + + assertTrue(group.hasOffsets) + assertTrue(group.allOffsets.isEmpty) + + group.completePendingTxnOffsetCommit(producerId, isCommit = true) + assertTrue(group.hasOffsets) + assertFalse(group.allOffsets.isEmpty) + assertEquals(offsets.get(foo0), group.offset(foo0.topicPartition)) + } + @Test def testExpireOffset(): Unit = { val memberId = "" diff --git a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala index b46abdc5cfd1c..4733c8ad9e4a8 100644 --- a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala @@ -24,7 +24,7 @@ import kafka.utils.{Logging, TestUtils} import scala.jdk.CollectionConverters._ import org.junit.jupiter.api.{BeforeEach, TestInfo} import com.yammer.metrics.core.Gauge -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -39,7 +39,7 @@ class MetricsDuringTopicCreationDeletionTest extends KafkaServerTestHarness with private val createDeleteIterations = 3 private val overridingProps = new Properties - overridingProps.put(KafkaConfig.DeleteTopicEnableProp, "true") + overridingProps.put(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "true") overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false") // speed up the test for UnderReplicatedPartitions, which relies on the ISR expiry thread to execute concurrently with topic creation // But the replica.lag.time.max.ms value still need to consider the slow Jenkins testing environment diff --git a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala index a0ba0c478c76b..cf1f1a0617782 100644 --- a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala @@ -372,6 +372,43 @@ class LocalLogTest { assertTrue(fetchDataInfo.records.records.asScala.isEmpty) } + @Test + def testWhenFetchOffsetHigherThanMaxOffset(): Unit = { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + for (offset <- 0 to 4) { + appendRecords(List(record), initialOffset = offset) + if (offset % 2 != 0) + log.roll() + } + assertEquals(3, log.segments.numberOfSegments) + + // case-0: valid case, `startOffset` < `maxOffsetMetadata.offset` + var fetchDataInfo = readRecords(startOffset = 3L, maxOffsetMetadata = new LogOffsetMetadata(4L, 4L, 0)) + assertEquals(1, fetchDataInfo.records.records.asScala.size) + assertEquals(new LogOffsetMetadata(3, 2L, 69), fetchDataInfo.fetchOffsetMetadata) + + // case-1: `startOffset` == `maxOffsetMetadata.offset` + fetchDataInfo = readRecords(startOffset = 4L, maxOffsetMetadata = new LogOffsetMetadata(4L, 4L, 0)) + assertTrue(fetchDataInfo.records.records.asScala.isEmpty) + assertEquals(new LogOffsetMetadata(4L, 4L, 0), fetchDataInfo.fetchOffsetMetadata) + + // case-2: `startOffset` > `maxOffsetMetadata.offset` + fetchDataInfo = readRecords(startOffset = 5L, maxOffsetMetadata = new LogOffsetMetadata(4L, 4L, 0)) + assertTrue(fetchDataInfo.records.records.asScala.isEmpty) + assertEquals(new LogOffsetMetadata(5L, 4L, 69), fetchDataInfo.fetchOffsetMetadata) + + // case-3: `startOffset` < `maxMessageOffset.offset` but `maxMessageOffset.messageOnlyOffset` is true + fetchDataInfo = readRecords(startOffset = 3L, maxOffsetMetadata = new LogOffsetMetadata(4L, -1L, -1)) + assertTrue(fetchDataInfo.records.records.asScala.isEmpty) + assertEquals(new LogOffsetMetadata(3L, 2L, 69), fetchDataInfo.fetchOffsetMetadata) + + // case-4: `startOffset` < `maxMessageOffset.offset`, `maxMessageOffset.messageOnlyOffset` is false, but + // `maxOffsetMetadata.segmentBaseOffset` < `startOffset.segmentBaseOffset` + fetchDataInfo = readRecords(startOffset = 3L, maxOffsetMetadata = new LogOffsetMetadata(4L, 0L, 40)) + assertTrue(fetchDataInfo.records.records.asScala.isEmpty) + assertEquals(new LogOffsetMetadata(3L, 2L, 69), fetchDataInfo.fetchOffsetMetadata) + } + @Test def testTruncateTo(): Unit = { for (offset <- 0 to 11) { diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala index 876082dd074ac..9f93c241ae39f 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala @@ -27,6 +27,7 @@ import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_11_0_IV0, IBP_0_9_0} +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import org.junit.jupiter.api.Assertions._ @@ -259,7 +260,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, cleanerConfig.dedupeBufferSize.toString) props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, cleanerConfig.dedupeBufferLoadFactor.toString) props.put(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, cleanerConfig.ioBufferSize.toString) - props.put(KafkaConfig.MessageMaxBytesProp, cleanerConfig.maxMessageSize.toString) + props.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, cleanerConfig.maxMessageSize.toString) props.put(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, cleanerConfig.backoffMs.toString) props.put(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, cleanerConfig.maxIoBytesPerSecond.toString) KafkaConfig.fromProps(props) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index beaa5f4c3f0dd..ee62b3f0f21a8 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.transaction.TransactionLogConfigs -import org.apache.kafka.server.metrics.KafkaMetricsGroup +import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanerConfig, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetMap, ProducerStateManager, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.utils.Throttler @@ -80,7 +80,6 @@ class LogCleanerTest extends Logging { logs = new Pool[TopicPartition, UnifiedLog](), logDirFailureChannel = new LogDirFailureChannel(1), time = time) - val metricsToVerify = new java.util.HashMap[String, java.util.List[java.util.Map[String, String]]]() logCleaner.cleanerManager.gaugeMetricNameWithTag.asScala.foreach { metricNameAndTags => val tags = new java.util.ArrayList[java.util.Map[String, String]]() @@ -120,6 +119,27 @@ class LogCleanerTest extends Logging { } } + @Test + def testMetricsActiveAfterReconfiguration(): Unit = { + val logCleaner = new LogCleaner(new CleanerConfig(true), + logDirs = Array(TestUtils.tempDir()), + logs = new Pool[TopicPartition, UnifiedLog](), + logDirFailureChannel = new LogDirFailureChannel(1), + time = time) + + try { + logCleaner.startup() + var nonexistent = LogCleaner.MetricNames.diff(KafkaYammerMetrics.defaultRegistry.allMetrics().keySet().asScala.map(_.getName)) + assertEquals(0, nonexistent.size, s"$nonexistent should be existent") + + logCleaner.reconfigure(new KafkaConfig(TestUtils.createBrokerConfig(1, "localhost:2181")), + new KafkaConfig(TestUtils.createBrokerConfig(1, "localhost:2181"))) + + nonexistent = LogCleaner.MetricNames.diff(KafkaYammerMetrics.defaultRegistry.allMetrics().keySet().asScala.map(_.getName)) + assertEquals(0, nonexistent.size, s"$nonexistent should be existent") + } finally logCleaner.shutdown() + } + /** * Test simple log cleaning */ diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 1b13225cc1478..2670d6e6f7736 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -21,13 +21,13 @@ import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM import org.apache.kafka.common.config.ConfigDef.Type.INT -import org.apache.kafka.common.config.{ConfigException, TopicConfig} +import org.apache.kafka.common.config.{ConfigException, SslConfigs, TopicConfig} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import java.util.{Collections, Properties} import org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1 -import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.storage.internals.log.{LogConfig, ThrottledReplicaListValidator} import org.junit.jupiter.params.ParameterizedTest @@ -188,7 +188,7 @@ class LogConfigTest { val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") kafkaProps.put("unknown.broker.password.config", "aaaaa") kafkaProps.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "50") - kafkaProps.put(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG, "somekeypassword") + kafkaProps.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, "somekeypassword") val kafkaConfig = KafkaConfig.fromProps(kafkaProps) val topicOverrides = new Properties // Only set as a topic config @@ -196,7 +196,7 @@ class LogConfigTest { // Overrides value from broker config topicOverrides.setProperty(TopicConfig.RETENTION_BYTES_CONFIG, "100") // Unknown topic config, but known broker config - topicOverrides.setProperty(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "sometrustpasswrd") + topicOverrides.setProperty(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "sometrustpasswrd") // Unknown config topicOverrides.setProperty("unknown.topic.password.config", "bbbb") // We don't currently have any sensitive topic configs, if we add them, we should set one here @@ -297,7 +297,7 @@ class LogConfigTest { props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs.toString) props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, localRetentionBytes.toString) assertThrows(classOf[ConfigException], - () => LogConfig.validate(props, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(props, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem())) } @Test @@ -309,17 +309,17 @@ class LogConfigTest { val logProps = new Properties() logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) logProps.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") - LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled) + LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem()) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem())) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "delete,compact") assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem())) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact,delete") assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem())) } @ParameterizedTest(name = "testEnableRemoteLogStorage with sysRemoteStorageEnabled: {0}") @@ -332,10 +332,10 @@ class LogConfigTest { val logProps = new Properties() logProps.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") if (sysRemoteStorageEnabled) { - LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled) + LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem()) } else { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem())) assertTrue(message.getMessage.contains("Tiered Storage functionality is disabled in the broker")) } } @@ -355,10 +355,10 @@ class LogConfigTest { logProps.put(TopicConfig.RETENTION_MS_CONFIG, "500") if (sysRemoteStorageEnabled) { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem())) assertTrue(message.getMessage.contains(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG)) } else { - LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled) + LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem()) } } @@ -377,10 +377,10 @@ class LogConfigTest { logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, "128") if (sysRemoteStorageEnabled) { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem())) assertTrue(message.getMessage.contains(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG)) } else { - LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled) + LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem()) } } @@ -395,10 +395,10 @@ class LogConfigTest { if (sysRemoteStorageEnabled) { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validateBrokerLogConfigValues(kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)) + () => LogConfig.validateBrokerLogConfigValues(kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem())) assertTrue(message.getMessage.contains(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG)) } else { - LogConfig.validateBrokerLogConfigValues(kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled) + LogConfig.validateBrokerLogConfigValues(kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.enableRemoteStorageSystem()) } } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index f8cc1f01efce9..c332aaa4c7e58 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -359,7 +359,7 @@ class LogLoaderTest { val wrapper = Mockito.spy(segment) Mockito.doAnswer { in => segmentsWithReads += wrapper - segment.read(in.getArgument(0, classOf[java.lang.Long]), in.getArgument(1, classOf[java.lang.Integer]), in.getArgument(2, classOf[java.lang.Long]), in.getArgument(3, classOf[java.lang.Boolean])) + segment.read(in.getArgument(0, classOf[java.lang.Long]), in.getArgument(1, classOf[java.lang.Integer]), in.getArgument(2, classOf[java.util.Optional[java.lang.Long]]), in.getArgument(3, classOf[java.lang.Boolean])) }.when(wrapper).read(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any()) Mockito.doAnswer { in => recoveredSegments += wrapper diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index decbc09382b1a..e2272941ab3d6 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.storage.internals.log._ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.CsvSource +import org.junit.jupiter.params.provider.{CsvSource, ValueSource} import java.io.{File, RandomAccessFile} import java.util.{Optional, OptionalLong} @@ -144,6 +144,34 @@ class LogSegmentTest { checkEquals(ms2.records.iterator, read.records.records.iterator) } + @ParameterizedTest(name = "testReadWhenNoMaxPosition minOneMessage = {0}") + @ValueSource(booleans = Array(true, false)) + def testReadWhenNoMaxPosition(minOneMessage: Boolean): Unit = { + val maxPosition: Optional[java.lang.Long] = Optional.empty() + val maxSize = 1 + val seg = createSegment(40) + val ms = records(50, "hello", "there") + seg.append(51, RecordBatch.NO_TIMESTAMP, -1L, ms) + // read before first offset + var read = seg.read(48, maxSize, maxPosition, minOneMessage) + assertEquals(new LogOffsetMetadata(48, 40, 0), read.fetchOffsetMetadata) + assertTrue(read.records.records().iterator().asScala.isEmpty) + // read at first offset + read = seg.read(50, maxSize, maxPosition, minOneMessage) + assertEquals(new LogOffsetMetadata(50, 40, 0), read.fetchOffsetMetadata) + assertTrue(read.records.records().iterator().asScala.isEmpty) + // read at last offset + read = seg.read(51, maxSize, maxPosition, minOneMessage) + assertEquals(new LogOffsetMetadata(51, 40, 39), read.fetchOffsetMetadata) + assertTrue(read.records.records().iterator().asScala.isEmpty) + // read at log-end-offset + read = seg.read(52, maxSize, maxPosition, minOneMessage) + assertNull(read) + // read beyond log-end-offset + read = seg.read(53, maxSize, maxPosition, minOneMessage) + assertNull(read) + } + /** * In a loop append two messages then truncate off the second of those messages and check that we can read * the first but not the second message. @@ -331,7 +359,7 @@ class LogSegmentTest { writeNonsenseToFile(indexFile, 5, indexFile.length.toInt) seg.recover(newProducerStateManager(), Optional.empty()) for (i <- 0 until 100) { - val records = seg.read(i, 1, seg.size(), true).records.records + val records = seg.read(i, 1, Optional.of(seg.size()), true).records.records assertEquals(i, records.iterator.next().offset) } } @@ -596,6 +624,17 @@ class LogSegmentTest { Utils.delete(tempDir) } + @Test + def testGetFirstBatchTimestamp(): Unit = { + val segment = createSegment(1) + assertEquals(Long.MaxValue, segment.getFirstBatchTimestamp) + + segment.append(1, 1000L, 1, MemoryRecords.withRecords(1, Compression.NONE, new SimpleRecord("one".getBytes))) + assertEquals(1000L, segment.getFirstBatchTimestamp) + + segment.close() + } + private def newProducerStateManager(): ProducerStateManager = { new ProducerStateManager( topicPartition, diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentsTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentsTest.scala index 134c32935cf09..a06adfb2e531e 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentsTest.scala @@ -27,6 +27,7 @@ import org.mockito.Mockito.{mock, when} import java.util.Arrays.asList import java.util.Optional +import java.util.OptionalLong import scala.jdk.CollectionConverters._ class LogSegmentsTest { @@ -106,7 +107,10 @@ class LogSegmentsTest { assertEquals(0, segments.numberOfSegments) assertFalse(segments.contains(offset1)) - segments.close() + // since we do segments.clear() before, we have to close segments one by one + seg1.close() + seg2.close() + seg3.close() } @Test @@ -127,6 +131,7 @@ class LogSegmentsTest { segments.add(seg) assertEntry(seg1, segments.firstEntry.get) assertEquals(Optional.of(seg1), segments.firstSegment) + assertEquals(OptionalLong.of(1), segments.firstSegmentBaseOffset()) assertEntry(seg, segments.lastEntry.get) assertEquals(Optional.of(seg), segments.lastSegment) } @@ -148,6 +153,13 @@ class LogSegmentsTest { assertEquals(Seq(), segments.values(4, 4).asScala.toSeq) assertEquals(Seq(seg4), segments.values(4, 5).asScala.toSeq) + // Test activeSegment + assertEquals(seg4, segments.activeSegment()) + + // Test nonActiveLogSegmentsFrom + assertEquals(Seq(seg2, seg3), segments.nonActiveLogSegmentsFrom(2).asScala.toSeq) + assertEquals(Seq(), segments.nonActiveLogSegmentsFrom(4).asScala.toSeq) + segments.close() } @@ -226,6 +238,8 @@ class LogSegmentsTest { val iterator = segments.higherSegments(9).iterator assertFalse(iterator.hasNext) } + + segments.close() } @Test @@ -238,5 +252,28 @@ class LogSegmentsTest { assertEquals(Int.MaxValue, LogSegments.sizeInBytes(asList(logSegment))) assertEquals(largeSize, LogSegments.sizeInBytes(asList(logSegment, logSegment))) assertTrue(UnifiedLog.sizeInBytes(asList(logSegment, logSegment)) > Int.MaxValue) + + val logSegments: LogSegments = new LogSegments(topicPartition) + logSegments.add(logSegment) + assertEquals(Int.MaxValue, logSegments.sizeInBytes()) + + logSegment.close() + } + + @Test + def testUpdateDir(): Unit = { + val seg1 = createSegment(1) + val segments = new LogSegments(topicPartition) + segments.add(seg1) + + val newDir: File = TestUtils.tempDir() + segments.updateParentDir(newDir) + assertEquals(newDir, seg1.log().file().getParentFile) + assertEquals(newDir, seg1.timeIndexFile().getParentFile) + assertEquals(newDir, seg1.offsetIndexFile().getParentFile) + assertEquals(newDir, seg1.txnIndex().file().getParentFile) + + seg1.close() + Utils.delete(newDir) } } diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 2394bc2d028aa..c18371854fe49 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -3246,7 +3246,7 @@ class UnifiedLogTest { val readInfo = segment.read(offsetMetadata.messageOffset, 2048, - segment.size, + Optional.of(segment.size), false) if (offsetMetadata.relativePositionInSegment < segment.size) @@ -4211,6 +4211,65 @@ class UnifiedLogTest { assertEquals(31, log.localLogStartOffset()) } + @Test + def testConvertToOffsetMetadataDoesNotThrowOffsetOutOfRangeError(): Unit = { + val logConfig = LogTestUtils.createLogConfig(localRetentionBytes = 1, fileDeleteDelayMs = 0, remoteLogStorageEnable = true) + val log = createLog(logDir, logConfig, remoteStorageSystemEnable = true) + + var offset = 0L + for(_ <- 0 until 50) { + val records = TestUtils.singletonRecords("test".getBytes()) + val info = log.appendAsLeader(records, leaderEpoch = 0) + offset = info.lastOffset + if (offset != 0 && offset % 10 == 0) + log.roll() + } + assertEquals(5, log.logSegments.size) + log.updateHighWatermark(log.logEndOffset) + // simulate calls to upload 3 segments to remote storage + log.updateHighestOffsetInRemoteStorage(30) + + log.deleteOldSegments() + assertEquals(2, log.logSegments.size()) + assertEquals(0, log.logStartOffset) + assertEquals(31, log.localLogStartOffset()) + + log.updateLogStartOffsetFromRemoteTier(15) + assertEquals(15, log.logStartOffset) + + // case-1: offset is higher than the local-log-start-offset. + // log-start-offset < local-log-start-offset < offset-to-be-converted < log-end-offset + assertEquals(new LogOffsetMetadata(35, 31, 288), log.maybeConvertToOffsetMetadata(35)) + // case-2: offset is less than the local-log-start-offset + // log-start-offset < offset-to-be-converted < local-log-start-offset < log-end-offset + assertEquals(new LogOffsetMetadata(29, -1L, -1), log.maybeConvertToOffsetMetadata(29)) + // case-3: offset is higher than the log-end-offset + // log-start-offset < local-log-start-offset < log-end-offset < offset-to-be-converted + assertEquals(new LogOffsetMetadata(log.logEndOffset + 1, -1L, -1), log.maybeConvertToOffsetMetadata(log.logEndOffset + 1)) + // case-4: offset is less than the log-start-offset + // offset-to-be-converted < log-start-offset < local-log-start-offset < log-end-offset + assertEquals(new LogOffsetMetadata(14, -1L, -1), log.maybeConvertToOffsetMetadata(14)) + } + + @Test + def testGetFirstBatchTimestampForSegments(): Unit = { + val log = createLog(logDir, LogTestUtils.createLogConfig()) + + val segments: java.util.List[LogSegment] = new java.util.ArrayList[LogSegment]() + val seg1 = LogTestUtils.createSegment(1, logDir, 10, Time.SYSTEM) + val seg2 = LogTestUtils.createSegment(2, logDir, 10, Time.SYSTEM) + segments.add(seg1) + segments.add(seg2) + assertEquals(Seq(Long.MaxValue, Long.MaxValue), log.getFirstBatchTimestampForSegments(segments).asScala.toSeq) + + seg1.append(1, 1000L, 1, MemoryRecords.withRecords(1, Compression.NONE, new SimpleRecord("one".getBytes))) + seg2.append(2, 2000L, 1, MemoryRecords.withRecords(2, Compression.NONE, new SimpleRecord("two".getBytes))) + assertEquals(Seq(1000L, 2000L), log.getFirstBatchTimestampForSegments(segments).asScala.toSeq) + + seg1.close() + seg2.close() + } + private def appendTransactionalToBuffer(buffer: ByteBuffer, producerId: Long, producerEpoch: Short, diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index ac8307bdf749c..dce053489f0d5 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -37,8 +37,8 @@ import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.utils._ import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.CredentialProvider -import org.apache.kafka.server.common.{Features, MetadataVersion} -import org.apache.kafka.server.config.QuotaConfigs +import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion} +import org.apache.kafka.server.config.{ServerConfigs, QuotaConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils} import org.apache.log4j.Level @@ -80,7 +80,7 @@ class SocketServerTest { TestUtils.clearYammerMetrics() private val apiVersionManager = new SimpleApiVersionManager(ListenerType.BROKER, true, false, - () => new Features(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, true)) + () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, true)) var server: SocketServer = _ val sockets = new ArrayBuffer[Socket] @@ -2051,7 +2051,7 @@ class SocketServerTest { val sslProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, interBrokerSecurityProtocol = Some(SecurityProtocol.SSL), trustStoreFile = Some(trustStoreFile)) sslProps.put(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:0") - sslProps.put(KafkaConfig.NumNetworkThreadsProp, "1") + sslProps.put(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, "1") sslProps } diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala index e6153bd6c4c3a..3416ffe65b690 100644 --- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala +++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.Time import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.ProcessRole import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -64,7 +64,7 @@ class RaftManagerTest { } props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") - props.setProperty(KafkaConfig.BrokerIdProp, nodeId.toString) + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, nodeId.toString) new KafkaConfig(props) } diff --git a/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala b/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala index f482d221763f3..c64235725bb62 100644 --- a/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala +++ b/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala @@ -27,6 +27,7 @@ import kafka.zk.KafkaZkClient import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclOperation} import org.apache.kafka.common.acl.AclOperation._ import org.apache.kafka.common.acl.AclPermissionType.ALLOW +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.resource.PatternType.LITERAL import org.apache.kafka.common.resource.ResourcePattern @@ -40,7 +41,7 @@ import org.apache.kafka.network.Session import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST import org.apache.kafka.security.authorizer.AuthorizerUtils import org.apache.kafka.server.authorizer._ -import org.apache.kafka.server.config.{Defaults, KafkaSecurityConfigs} +import org.apache.kafka.server.config.DelegationTokenManagerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -59,8 +60,8 @@ class DelegationTokenManagerTest extends QuorumTestHarness { val tokenManagers = mutable.Buffer[DelegationTokenManager]() val secretKey = "secretKey" - val maxLifeTimeMsDefault = Defaults.DELEGATION_TOKEN_MAX_LIFE_TIME_MS - val renewTimeMsDefault = Defaults.DELEGATION_TOKEN_EXPIRY_TIME_MS + val maxLifeTimeMsDefault = DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFE_TIME_MS_DEFAULT + val renewTimeMsDefault = DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_DEFAULT var tokenCache: DelegationTokenCache = _ var props: Properties = _ @@ -72,8 +73,8 @@ class DelegationTokenManagerTest extends QuorumTestHarness { override def setUp(testInfo: TestInfo): Unit = { super.setUp(testInfo) props = TestUtils.createBrokerConfig(0, zkConnect, enableToken = true) - props.put(KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, ScramMechanism.mechanismNames().asScala.mkString(",")) - props.put(KafkaConfig.DelegationTokenSecretKeyProp, secretKey) + props.put(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, ScramMechanism.mechanismNames().asScala.mkString(",")) + props.put(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, secretKey) tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames()) } diff --git a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala index 30318c1ccd967..b79cf12aa4b33 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.record.RecordVersion import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse, RequestUtils} import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.common.{GroupVersion, MetadataVersion} import org.apache.kafka.test.TestUtils import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Tag @@ -69,9 +69,12 @@ abstract class AbstractApiVersionsRequestTest(cluster: ClusterInstance) { assertEquals(MetadataVersion.latestTesting().featureLevel(), apiVersionsResponse.data().finalizedFeatures().find(MetadataVersion.FEATURE_NAME).minVersionLevel()) assertEquals(MetadataVersion.latestTesting().featureLevel(), apiVersionsResponse.data().finalizedFeatures().find(MetadataVersion.FEATURE_NAME).maxVersionLevel()) - assertEquals(1, apiVersionsResponse.data().supportedFeatures().size()) + assertEquals(2, apiVersionsResponse.data().supportedFeatures().size()) assertEquals(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), apiVersionsResponse.data().supportedFeatures().find(MetadataVersion.FEATURE_NAME).minVersion()) assertEquals(MetadataVersion.latestTesting().featureLevel(), apiVersionsResponse.data().supportedFeatures().find(MetadataVersion.FEATURE_NAME).maxVersion()) + + assertEquals(0, apiVersionsResponse.data().supportedFeatures().find(GroupVersion.FEATURE_NAME).minVersion()) + assertEquals(GroupVersion.GV_1.featureLevel(), apiVersionsResponse.data().supportedFeatures().find(GroupVersion.FEATURE_NAME).maxVersion()) } val expectedApis = if (!cluster.isKRaftTest) { ApiVersionsResponse.collectApis( diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala index 11b7ceb2df4b8..f9750618ae0ea 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala @@ -33,7 +33,6 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, Test} import org.mockito.Mockito.{mock, verify, when} -import java.util.Optional import scala.collection.{Map, Set, mutable} import scala.jdk.CollectionConverters._ @@ -313,12 +312,10 @@ class AbstractFetcherManagerTest { override def fetchEarliestLocalOffset(topicPartition: TopicPartition, currentLeaderEpoch: Int): OffsetAndEpoch = new OffsetAndEpoch(1L, 0) } - private class MockResizeFetcherTierStateMachine extends TierStateMachine { + private class MockResizeFetcherTierStateMachine extends TierStateMachine(null, null, false) { override def start(topicPartition: TopicPartition, currentFetchState: PartitionFetchState, fetchPartitionData: PartitionData): PartitionFetchState = { throw new UnsupportedOperationException("Materializing tier state is not supported in this test.") } - - override def maybeAdvanceState(tp: TopicPartition, currentFetchState: PartitionFetchState): Optional[PartitionFetchState] = Optional.empty[PartitionFetchState] } private class TestResizeFetcherThread(sourceBroker: BrokerEndPoint, failedPartitions: FailedPartitions, fetchTierStateMachine: TierStateMachine) diff --git a/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala index e35561ef63990..ebe4bd05f3ffb 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala @@ -24,7 +24,7 @@ import org.apache.kafka.common.message.MetadataRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.server.config.ReplicationConfigs +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs} import org.junit.jupiter.api.Assertions.assertEquals abstract class AbstractMetadataRequestTest extends BaseRequestTest { @@ -32,7 +32,7 @@ abstract class AbstractMetadataRequestTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2") - properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") + properties.setProperty(ServerConfigs.BROKER_RACK_CONFIG, s"rack/${properties.getProperty(ServerConfigs.BROKER_ID_CONFIG)}") } protected def requestData(topics: List[String], allowAutoTopicCreation: Boolean): MetadataRequestData = { diff --git a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala index 4449857c7b20d..ce83b95526118 100644 --- a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala @@ -18,11 +18,12 @@ package kafka.server import kafka.network.SocketServer import org.apache.kafka.clients.admin.ScramMechanism +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AlterUserScramCredentialsRequest, AlterUserScramCredentialsResponse} -import org.apache.kafka.server.config.KafkaSecurityConfigs +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -36,9 +37,9 @@ import scala.jdk.CollectionConverters._ class AlterUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.ControlledShutdownEnableProp, "false") - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AlterCredentialsTest.TestAuthorizer].getName) - properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName) + properties.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestAuthorizer].getName) + properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName) } private val user1 = "user1" diff --git a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala index b84f6761f5968..b7ce044b85384 100644 --- a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala @@ -25,6 +25,7 @@ import kafka.network.SocketServer import kafka.security.authorizer.AclAuthorizer import org.apache.kafka.metadata.authorizer.StandardAuthorizer import org.apache.kafka.clients.admin.ScramMechanism +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult import org.apache.kafka.common.message.{AlterUserScramCredentialsRequestData, DescribeUserScramCredentialsRequestData} @@ -34,8 +35,8 @@ import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrinci import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult} import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.config.KafkaSecurityConfigs -import org.junit.jupiter.api.{Test, BeforeEach, TestInfo} +import org.apache.kafka.server.config.ServerConfigs +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -55,16 +56,16 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { if (TestInfoUtils.isKRaft(testInfo)) { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) if (testInfo.getDisplayName.contains("quorum=kraft-IBP_3_4")) { testMetadataVersion = MetadataVersion.IBP_3_4_IV0 } } else { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[AlterCredentialsTest.TestAuthorizer].getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestAuthorizer].getName) } - this.serverConfig.setProperty(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName) - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") + this.serverConfig.setProperty(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName) + this.serverConfig.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") super.setUp(testInfo) } diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala index 0f593ad43e6ec..3a69669d349e5 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala @@ -43,18 +43,18 @@ object ApiVersionsRequestTest { def testApiVersionsRequestTemplate(): java.util.List[ClusterConfig] = { val serverProperties: java.util.HashMap[String, String] = controlPlaneListenerProperties() serverProperties.put("unstable.api.versions.enable", "false") - serverProperties.put("unstable.metadata.versions.enable", "true") + serverProperties.put("unstable.feature.versions.enable", "true") List(ClusterConfig.defaultBuilder() .setTypes(java.util.Collections.singleton(Type.ZK)) .setServerProperties(serverProperties) - .setMetadataVersion(MetadataVersion.IBP_3_8_IV0) + .setMetadataVersion(MetadataVersion.IBP_4_0_IVO) .build()).asJava } def testApiVersionsRequestIncludesUnreleasedApisTemplate(): java.util.List[ClusterConfig] = { val serverProperties: java.util.HashMap[String, String] = controlPlaneListenerProperties() serverProperties.put("unstable.api.versions.enable", "true") - serverProperties.put("unstable.metadata.versions.enable", "true") + serverProperties.put("unstable.feature.versions.enable", "true") List(ClusterConfig.defaultBuilder() .setTypes(java.util.Collections.singleton(Type.ZK)) .setServerProperties(serverProperties) @@ -64,7 +64,7 @@ object ApiVersionsRequestTest { def testApiVersionsRequestValidationV0Template(): java.util.List[ClusterConfig] = { val serverProperties: java.util.HashMap[String, String] = controlPlaneListenerProperties() serverProperties.put("unstable.api.versions.enable", "false") - serverProperties.put("unstable.metadata.versions.enable", "false") + serverProperties.put("unstable.feature.versions.enable", "false") List(ClusterConfig.defaultBuilder() .setTypes(java.util.Collections.singleton(Type.ZK)) .setMetadataVersion(MetadataVersion.IBP_3_7_IV4) @@ -83,9 +83,9 @@ object ApiVersionsRequestTest { class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersionsRequestTest(cluster) { @ClusterTemplate("testApiVersionsRequestTemplate") - @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), metadataVersion = MetadataVersion.IBP_3_8_IV0, serverProperties = Array( + @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), metadataVersion = MetadataVersion.IBP_4_0_IVO, serverProperties = Array( new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "false"), - new ClusterConfigProperty(key = "unstable.metadata.versions.enable", value = "true") + new ClusterConfigProperty(key = "unstable.feature.versions.enable", value = "true") )) def testApiVersionsRequest(): Unit = { val request = new ApiVersionsRequest.Builder().build() @@ -95,8 +95,8 @@ class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersio @ClusterTemplate("testApiVersionsRequestIncludesUnreleasedApisTemplate") @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "false"), - new ClusterConfigProperty(key = "unstable.metadata.versions.enable", value = "true"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true"), + new ClusterConfigProperty(key = "unstable.feature.versions.enable", value = "true"), )) def testApiVersionsRequestIncludesUnreleasedApis(): Unit = { val request = new ApiVersionsRequest.Builder().build() @@ -134,7 +134,7 @@ class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersio @ClusterTemplate("testApiVersionsRequestValidationV0Template") @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), metadataVersion = MetadataVersion.IBP_3_7_IV4, serverProperties = Array( new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "false"), - new ClusterConfigProperty(key = "unstable.metadata.versions.enable", value = "false"), + new ClusterConfigProperty(key = "unstable.feature.versions.enable", value = "false"), )) def testApiVersionsRequestValidationV0(): Unit = { val apiVersionsRequest = new ApiVersionsRequest.Builder().build(0.asInstanceOf[Short]) diff --git a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala index 9ee92469e53f9..d2bb8ea715dd5 100644 --- a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala @@ -38,6 +38,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, KafkaPrincipalSerd import org.apache.kafka.common.utils.{SecurityUtils, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig} +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue} import org.junit.jupiter.api.{BeforeEach, Test} @@ -65,7 +66,7 @@ class AutoTopicCreationManagerTest { @BeforeEach def setup(): Unit = { val props = TestUtils.createBrokerConfig(1, "localhost") - props.setProperty(KafkaConfig.RequestTimeoutMsProp, requestTimeout.toString) + props.setProperty(ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG, requestTimeout.toString) props.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, internalTopicPartitions.toString) props.setProperty(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, internalTopicPartitions.toString) diff --git a/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala index f17b27e1ee54a..cb42739551601 100644 --- a/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala @@ -24,6 +24,7 @@ import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.record.Record import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.apache.kafka.common.serialization.StringSerializer +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.AfterEach import java.util @@ -36,7 +37,7 @@ class BaseFetchRequestTest extends BaseRequestTest { protected var producer: KafkaProducer[String, String] = _ override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.FetchMaxBytes, Int.MaxValue.toString) + properties.put(ServerConfigs.FETCH_MAX_BYTES_CONFIG, Int.MaxValue.toString) } @AfterEach diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala index eddeeb4253d5e..4c1494380ea60 100644 --- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -24,6 +24,7 @@ import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, ResponseHeader} import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.BrokerState +import org.apache.kafka.server.config.ServerConfigs import java.io.{DataInputStream, DataOutputStream} import java.net.Socket @@ -43,7 +44,7 @@ abstract class BaseRequestTest extends IntegrationTestHarness { override def modifyConfigs(props: Seq[Properties]): Unit = { props.foreach { p => - p.put(KafkaConfig.ControlledShutdownEnableProp, "false") + p.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") brokerPropertyOverrides(p) } } diff --git a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala index 78432b7110580..9c43df2b2d258 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala @@ -18,7 +18,7 @@ package kafka.server import org.apache.kafka.common.feature.{Features, SupportedVersionRange} -import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.common.{Features => ServerFeatures, MetadataVersion} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.Test @@ -95,6 +95,7 @@ class BrokerFeaturesTest { val expectedFeatures = Map[String, Short]( MetadataVersion.FEATURE_NAME -> MetadataVersion.latestTesting().featureLevel(), + ServerFeatures.GROUP_VERSION.featureName() -> ServerFeatures.GROUP_VERSION.latestTesting(), "test_feature_1" -> 4, "test_feature_2" -> 3, "test_feature_3" -> 7) diff --git a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala index 7752f0dae2249..b0162dc635842 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala @@ -232,11 +232,11 @@ class BrokerLifecycleManagerTest { poll(ctx, manager, prepareResponse[BrokerHeartbeatRequest](ctx, new BrokerHeartbeatResponse(new BrokerHeartbeatResponseData()))) .data().offlineLogDirs().asScala.map(_.toString).toSet assertEquals(Set.empty, nextHeartbeatDirs()) - manager.propagateDirectoryFailure(Uuid.fromString("h3sC4Yk-Q9-fd0ntJTocCA")) + manager.propagateDirectoryFailure(Uuid.fromString("h3sC4Yk-Q9-fd0ntJTocCA"), Integer.MAX_VALUE) assertEquals(Set("h3sC4Yk-Q9-fd0ntJTocCA"), nextHeartbeatDirs()) - manager.propagateDirectoryFailure(Uuid.fromString("ej8Q9_d2Ri6FXNiTxKFiow")) + manager.propagateDirectoryFailure(Uuid.fromString("ej8Q9_d2Ri6FXNiTxKFiow"), Integer.MAX_VALUE) assertEquals(Set("h3sC4Yk-Q9-fd0ntJTocCA", "ej8Q9_d2Ri6FXNiTxKFiow"), nextHeartbeatDirs()) - manager.propagateDirectoryFailure(Uuid.fromString("1iF76HVNRPqC7Y4r6647eg")) + manager.propagateDirectoryFailure(Uuid.fromString("1iF76HVNRPqC7Y4r6647eg"), Integer.MAX_VALUE) assertEquals(Set("h3sC4Yk-Q9-fd0ntJTocCA", "ej8Q9_d2Ri6FXNiTxKFiow", "1iF76HVNRPqC7Y4r6647eg"), nextHeartbeatDirs()) manager.close() } @@ -285,7 +285,7 @@ class BrokerLifecycleManagerTest { assertEquals(1000L, manager.brokerEpoch) // Trigger JBOD MV update - manager.handleKraftJBODMetadataVersionUpdate() + manager.resendBrokerRegistrationUnlessZkMode() // Accept new registration, response sets epoch to 1200 nextRegistrationRequest(1200L) diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestsTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestsTest.scala index b2e4e0f85f1d2..b1f8b8405e743 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestsTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestsTest.scala @@ -18,7 +18,7 @@ package kafka.server import kafka.server.GroupCoordinatorBaseRequestTest import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.common.ConsumerGroupState @@ -26,6 +26,7 @@ import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData.{Assign import org.apache.kafka.common.message.{ConsumerGroupDescribeRequestData, ConsumerGroupDescribeResponseData, ConsumerGroupHeartbeatResponseData} import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse} +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.extension.ExtendWith import org.junit.jupiter.api.{Tag, Timeout} @@ -60,11 +61,46 @@ class ConsumerGroupDescribeRequestsTest(cluster: ClusterInstance) extends GroupC assertEquals(expectedResponse, consumerGroupDescribeResponse.data) } - @ClusterTest(types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + types = Array(Type.KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ) + def testConsumerGroupDescribeIsInaccessibleWhenDisabledByGroupVersion(): Unit = { + val consumerGroupDescribeRequest = new ConsumerGroupDescribeRequest.Builder( + new ConsumerGroupDescribeRequestData().setGroupIds(List("grp-1", "grp-2").asJava) + ).build(ApiKeys.CONSUMER_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled)) + + val consumerGroupDescribeResponse = connectAndReceive[ConsumerGroupDescribeResponse](consumerGroupDescribeRequest) + val expectedResponse = new ConsumerGroupDescribeResponseData() + expectedResponse.groups().add( + new ConsumerGroupDescribeResponseData.DescribedGroup() + .setGroupId("grp-1") + .setErrorCode(Errors.UNSUPPORTED_VERSION.code) + ) + expectedResponse.groups.add( + new ConsumerGroupDescribeResponseData.DescribedGroup() + .setGroupId("grp-2") + .setErrorCode(Errors.UNSUPPORTED_VERSION.code) + ) + + assertEquals(expectedResponse, consumerGroupDescribeResponse.data) + } + + @ClusterTest( + types = Array(Type.KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testConsumerGroupDescribeWithNewGroupCoordinator(): Unit = { // Creates the __consumer_offsets topics because it won't be created automatically // in this test because it does not use FindCoordinator API. diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala index 16e0f3d0ef0ee..1cf0cedaf23b7 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala @@ -17,13 +17,14 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, Type} import kafka.test.junit.ClusterTestExtensions import kafka.test.junit.RaftClusterInvocationContext.RaftClusterInstance import kafka.utils.TestUtils import org.apache.kafka.common.message.{ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse} +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertNotNull} import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.extension.ExtendWith @@ -36,7 +37,7 @@ import scala.jdk.CollectionConverters._ class ConsumerGroupHeartbeatRequestTest(cluster: ClusterInstance) { @ClusterTest() - def testConsumerGroupHeartbeatIsAccessibleWhenEnabled(): Unit = { + def testConsumerGroupHeartbeatIsInaccessibleWhenDisabledByStaticConfig(): Unit = { val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequest.Builder( new ConsumerGroupHeartbeatRequestData() ).build() @@ -46,11 +47,35 @@ class ConsumerGroupHeartbeatRequestTest(cluster: ClusterInstance) { assertEquals(expectedResponse, consumerGroupHeartbeatResponse.data) } - @ClusterTest(types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + types = Array(Type.KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ) + def testConsumerGroupHeartbeatIsInaccessibleWhenDisabledByGroupVersion(): Unit = { + val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequest.Builder( + new ConsumerGroupHeartbeatRequestData() + ).build() + + val consumerGroupHeartbeatResponse = connectAndReceive(consumerGroupHeartbeatRequest) + val expectedResponse = new ConsumerGroupHeartbeatResponseData().setErrorCode(Errors.UNSUPPORTED_VERSION.code) + assertEquals(expectedResponse, consumerGroupHeartbeatResponse.data) + } + + @ClusterTest( + types = Array(Type.KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testConsumerGroupHeartbeatIsAccessibleWhenNewGroupCoordinatorIsEnabled(): Unit = { val raftCluster = cluster.asInstanceOf[RaftClusterInstance] val admin = cluster.createAdminClient() @@ -134,11 +159,17 @@ class ConsumerGroupHeartbeatRequestTest(cluster: ClusterInstance) { assertEquals(-1, consumerGroupHeartbeatResponse.data.memberEpoch) } - @ClusterTest(types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + types = Array(Type.KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testRejoiningStaticMemberGetsAssignmentsBackWhenNewGroupCoordinatorIsEnabled(): Unit = { val raftCluster = cluster.asInstanceOf[RaftClusterInstance] val admin = cluster.createAdminClient() @@ -248,13 +279,19 @@ class ConsumerGroupHeartbeatRequestTest(cluster: ClusterInstance) { assertNotEquals(oldMemberId, consumerGroupHeartbeatResponse.data.memberId) } - @ClusterTest(types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "5000"), - new ClusterConfigProperty(key = "group.consumer.min.session.timeout.ms", value = "5000") - )) + @ClusterTest( + types = Array(Type.KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "5000"), + new ClusterConfigProperty(key = "group.consumer.min.session.timeout.ms", value = "5000") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testStaticMemberRemovedAfterSessionTimeoutExpiryWhenNewGroupCoordinatorIsEnabled(): Unit = { val raftCluster = cluster.asInstanceOf[RaftClusterInstance] val admin = cluster.createAdminClient() diff --git a/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala index 969e3e9a0db56..a93cb68be573e 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala @@ -17,13 +17,14 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions import org.apache.kafka.common.message.ListGroupsResponseData import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.coordinator.group.Group import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Tag import org.junit.jupiter.api.Timeout @@ -34,11 +35,16 @@ import org.junit.jupiter.api.extension.ExtendWith @ClusterTestDefaults(types = Array(Type.KRAFT)) @Tag("integration") class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testUpgradeFromEmptyClassicToConsumerGroup(): Unit = { // Creates the __consumer_offsets topics because it won't be created automatically // in this test because it does not use FindCoordinator API. @@ -103,11 +109,16 @@ class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoord ) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testDowngradeFromEmptyConsumerToClassicGroup(): Unit = { // Creates the __consumer_offsets topics because it won't be created automatically // in this test because it does not use FindCoordinator API. @@ -165,11 +176,16 @@ class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoord ) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testUpgradeFromSimpleGroupToConsumerGroup(): Unit = { // Creates the __consumer_offsets topics because it won't be created automatically // in this test because it does not use FindCoordinator API. diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 84a58278a42f3..a47ec6f7de445 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -53,8 +53,8 @@ import org.apache.kafka.controller.{Controller, ControllerRequestContext, Result import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer} -import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion, ProducerIdsBlock} -import org.apache.kafka.server.config.KRaftConfigs +import org.apache.kafka.server.common.{ApiMessageAndVersion, FinalizedFeatures, MetadataVersion, ProducerIdsBlock} +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs} import org.apache.kafka.server.util.FutureUtils import org.apache.kafka.storage.internals.log.CleanerConfig import org.junit.jupiter.api.Assertions._ @@ -169,7 +169,7 @@ class ControllerApisTest { ListenerType.CONTROLLER, true, false, - () => Features.fromKRaftVersion(MetadataVersion.latestTesting())), + () => FinalizedFeatures.fromKRaftVersion(MetadataVersion.latestTesting())), metadataCache ) } @@ -890,7 +890,7 @@ class ControllerApisTest { val controller = new MockController.Builder(). newInitialTopic("foo", fooId).build() val props = new Properties() - props.put(KafkaConfig.DeleteTopicEnableProp, "false") + props.put(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "false") controllerApis = createControllerApis(None, controller, props) val request = new DeleteTopicsRequestData() request.topics().add(new DeleteTopicState().setName("foo").setTopicId(ZERO_UUID)) diff --git a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala index f4b7f6bcd8d5d..86a32d126c2f7 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala @@ -18,6 +18,7 @@ import java.util.concurrent.ExecutionException import java.util.concurrent.TimeUnit import kafka.server.ClientQuotaManager.DefaultTags import kafka.utils.TestUtils +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.internals.KafkaFutureImpl import org.apache.kafka.common.message.CreatePartitionsRequestData import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic @@ -40,7 +41,7 @@ import org.apache.kafka.common.requests.DeleteTopicsResponse import org.apache.kafka.common.security.auth.AuthenticationContext import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder -import org.apache.kafka.server.config.{KafkaSecurityConfigs, QuotaConfigs} +import org.apache.kafka.server.config.{QuotaConfigs, ServerConfigs} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.test.{TestUtils => JTestUtils} import org.junit.jupiter.api.Assertions.assertEquals @@ -95,10 +96,10 @@ class ControllerMutationQuotaTest extends BaseRequestTest { override def brokerCount: Int = 1 override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.ControlledShutdownEnableProp, "false") + properties.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1") properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") - properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, + properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[ControllerMutationQuotaTest.TestPrincipalBuilder].getName) // Specify number of samples and window size. properties.put(QuotaConfigs.NUM_CONTROLLER_QUOTA_SAMPLES_CONFIG, ControllerQuotaSamples.toString) diff --git a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala index 1ef6e37720204..6081a0fe3d0c8 100644 --- a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala @@ -24,6 +24,7 @@ import org.apache.kafka.common.errors.InvalidPrincipalTypeException import org.apache.kafka.common.errors.DelegationTokenNotFoundException import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.SecurityUtils +import org.apache.kafka.server.config.DelegationTokenManagerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -43,11 +44,11 @@ class DelegationTokenRequestsTest extends IntegrationTestHarness with SaslSetup override def brokerCount = 1 - this.serverConfig.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "testKey") - this.controllerConfig.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "testKey") + this.serverConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "testKey") + this.controllerConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "testKey") // Remove expired tokens every minute. - this.serverConfig.setProperty(KafkaConfig.DelegationTokenExpiryCheckIntervalMsProp, "5000") - this.controllerConfig.setProperty(KafkaConfig.DelegationTokenExpiryCheckIntervalMsProp, "5000") + this.serverConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG, "5000") + this.controllerConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG, "5000") @BeforeEach override def setUp(testInfo: TestInfo): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala index 10b94c882c341..49bbee71a4f27 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala @@ -17,11 +17,12 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.coordinator.group.classic.ClassicGroupState +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.{assertEquals, fail} import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.extension.ExtendWith @@ -31,31 +32,44 @@ import org.junit.jupiter.api.extension.ExtendWith @ClusterTestDefaults(types = Array(Type.KRAFT)) @Tag("integration") class DeleteGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testDeleteGroupsWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testDeleteGroups(true) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testDeleteGroupsWithOldConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testDeleteGroups(false) } - @ClusterTest(types = Array(Type.ZK, Type.KRAFT, Type.CO_KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + types = Array(Type.ZK, Type.KRAFT, Type.CO_KRAFT), + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ) def testDeleteGroupsWithOldConsumerGroupProtocolAndOldGroupCoordinator(): Unit = { testDeleteGroups(false) } diff --git a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala index e377256ccc67b..91ea2e8840e42 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala @@ -18,12 +18,12 @@ package kafka.server import java.util.Collections - import kafka.utils._ import org.apache.kafka.common.message.DeleteTopicsRequestData import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{DeleteTopicsRequest, DeleteTopicsResponse} +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -34,7 +34,7 @@ class DeleteTopicsRequestWithDeletionDisabledTest extends BaseRequestTest { override def kraftControllerConfigs() = { val props = super.kraftControllerConfigs() - props.head.setProperty(KafkaConfig.DeleteTopicEnableProp, "false") + props.head.setProperty(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "false") props } diff --git a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala index eb6acd8d73e57..0148d29061e80 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.resource.ResourceType import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.ReplicationConfigs +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -40,7 +40,7 @@ class DescribeClusterRequestTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2") - properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") + properties.setProperty(ServerConfigs.BROKER_RACK_CONFIG, s"rack/${properties.getProperty(ServerConfigs.BROKER_ID_CONFIG)}") } @BeforeEach diff --git a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala index e8a84048b1341..b39364f01fc36 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala @@ -17,11 +17,12 @@ package kafka.server import kafka.network.SocketServer +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{DescribeUserScramCredentialsRequest, DescribeUserScramCredentialsResponse} import org.apache.kafka.metadata.authorizer.StandardAuthorizer -import org.apache.kafka.server.config.KafkaSecurityConfigs +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -33,13 +34,13 @@ import java.util.Properties */ class DescribeUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.ControlledShutdownEnableProp, "false") + properties.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") if (isKRaftTest()) { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) } else { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[DescribeCredentialsTest.TestAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[DescribeCredentialsTest.TestAuthorizer].getName) } - properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[DescribeCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName) + properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[DescribeCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName) } @ParameterizedTest diff --git a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala index a70cb9cfeeb9e..1a3d0e193e5e7 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala @@ -20,6 +20,7 @@ import java.util import kafka.utils.TestInfoUtils import kafka.network.SocketServer import kafka.security.authorizer.AclAuthorizer +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.metadata.authorizer.StandardAuthorizer import org.apache.kafka.common.message.{DescribeUserScramCredentialsRequestData, DescribeUserScramCredentialsResponseData} import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData.UserName @@ -28,8 +29,8 @@ import org.apache.kafka.common.requests.{DescribeUserScramCredentialsRequest, De import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal} import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult} -import org.apache.kafka.server.config.KafkaSecurityConfigs -import org.junit.jupiter.api.{Test, BeforeEach, TestInfo} +import org.apache.kafka.server.config.ServerConfigs +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -45,13 +46,13 @@ class DescribeUserScramCredentialsRequestTest extends BaseRequestTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { if (TestInfoUtils.isKRaft(testInfo)) { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) } else { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[AlterCredentialsTest.TestAuthorizer].getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestAuthorizer].getName) } - this.serverConfig.setProperty(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName) - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") + this.serverConfig.setProperty(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName) + this.serverConfig.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") super.setUp(testInfo) } diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 945d17b69c0a7..328d6e41b5722 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -30,14 +30,14 @@ import kafka.zk.KafkaZkClient import org.apache.kafka.common.{Endpoint, Reconfigurable} import org.apache.kafka.common.acl.{AclBinding, AclBindingFilter} import org.apache.kafka.common.config.types.Password -import org.apache.kafka.common.config.{ConfigException, SslConfigs} +import org.apache.kafka.common.config.{ConfigException, SaslConfigs, SslConfigs} import org.apache.kafka.common.metrics.{JmxReporter, Metrics} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.PasswordEncoderConfigs import org.apache.kafka.server.authorizer._ -import org.apache.kafka.server.config.{Defaults, KRaftConfigs, KafkaSecurityConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.{KafkaYammerMetrics, MetricConfigs} import org.apache.kafka.server.util.KafkaScheduler @@ -86,7 +86,7 @@ class DynamicBrokerConfigTest { assertEquals(newKeystore, config.originalsWithPrefix("listener.name.external.").get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) - assertEquals(oldKeystore, config.getString(KafkaSecurityConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) + assertEquals(oldKeystore, config.getString(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) assertEquals(oldKeystore, config.originals.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) assertEquals(oldKeystore, config.values.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) assertEquals(oldKeystore, config.originalsStrings.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) @@ -134,11 +134,11 @@ class DynamicBrokerConfigTest { @Test def testUpdateDynamicThreadPool(): Unit = { val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - origProps.put(KafkaConfig.NumIoThreadsProp, "4") - origProps.put(KafkaConfig.NumNetworkThreadsProp, "2") + origProps.put(ServerConfigs.NUM_IO_THREADS_CONFIG, "4") + origProps.put(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, "2") origProps.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "1") origProps.put(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, "1") - origProps.put(KafkaConfig.BackgroundThreadsProp, "3") + origProps.put(ServerConfigs.BACKGROUND_THREADS_CONFIG, "3") val config = KafkaConfig(origProps) val serverMock = Mockito.mock(classOf[KafkaBroker]) @@ -165,18 +165,18 @@ class DynamicBrokerConfigTest { val props = new Properties() - props.put(KafkaConfig.NumIoThreadsProp, "8") + props.put(ServerConfigs.NUM_IO_THREADS_CONFIG, "8") config.dynamicConfig.updateDefaultConfig(props) assertEquals(8, config.numIoThreads) Mockito.verify(handlerPoolMock).resizeThreadPool(newSize = 8) - props.put(KafkaConfig.NumNetworkThreadsProp, "4") + props.put(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, "4") config.dynamicConfig.updateDefaultConfig(props) assertEquals(4, config.numNetworkThreads) val captor: ArgumentCaptor[JMap[String, String]] = ArgumentCaptor.forClass(classOf[JMap[String, String]]) Mockito.verify(acceptorMock).reconfigure(captor.capture()) - assertTrue(captor.getValue.containsKey(KafkaConfig.NumNetworkThreadsProp)) - assertEquals(4, captor.getValue.get(KafkaConfig.NumNetworkThreadsProp)) + assertTrue(captor.getValue.containsKey(ServerConfigs.NUM_NETWORK_THREADS_CONFIG)) + assertEquals(4, captor.getValue.get(ServerConfigs.NUM_NETWORK_THREADS_CONFIG)) props.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "2") config.dynamicConfig.updateDefaultConfig(props) @@ -188,7 +188,7 @@ class DynamicBrokerConfigTest { assertEquals(2, config.numRecoveryThreadsPerDataDir) Mockito.verify(logManagerMock).resizeRecoveryThreadPool(newSize = 2) - props.put(KafkaConfig.BackgroundThreadsProp, "6") + props.put(ServerConfigs.BACKGROUND_THREADS_CONFIG, "6") config.dynamicConfig.updateDefaultConfig(props) assertEquals(6, config.backgroundThreads) Mockito.verify(schedulerMock).resizeThreadPool(6) @@ -263,7 +263,7 @@ class DynamicBrokerConfigTest { def testReconfigurableValidation(): Unit = { val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val config = KafkaConfig(origProps) - val invalidReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, KafkaConfig.BrokerIdProp, "some.prop") + val invalidReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, ServerConfigs.BROKER_ID_CONFIG, "some.prop") val validReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "some.prop") def createReconfigurable(configs: Set[String]) = new Reconfigurable { @@ -386,7 +386,7 @@ class DynamicBrokerConfigTest { props.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, "config-encoder-secret") val configWithSecret = KafkaConfig(props) val dynamicProps = new Properties - dynamicProps.put(KafkaSecurityConfigs.SASL_JAAS_CONFIG, "myLoginModule required;") + dynamicProps.put(SaslConfigs.SASL_JAAS_CONFIG, "myLoginModule required;") try { configWithoutSecret.dynamicConfig.toPersistentProps(dynamicProps, perBrokerConfig = true) @@ -394,46 +394,46 @@ class DynamicBrokerConfigTest { case _: ConfigException => // expected exception } val persistedProps = configWithSecret.dynamicConfig.toPersistentProps(dynamicProps, perBrokerConfig = true) - assertFalse(persistedProps.getProperty(KafkaSecurityConfigs.SASL_JAAS_CONFIG).contains("myLoginModule"), + assertFalse(persistedProps.getProperty(SaslConfigs.SASL_JAAS_CONFIG).contains("myLoginModule"), "Password not encoded") val decodedProps = configWithSecret.dynamicConfig.fromPersistentProps(persistedProps, perBrokerConfig = true) - assertEquals("myLoginModule required;", decodedProps.getProperty(KafkaSecurityConfigs.SASL_JAAS_CONFIG)) + assertEquals("myLoginModule required;", decodedProps.getProperty(SaslConfigs.SASL_JAAS_CONFIG)) } @Test def testPasswordConfigEncoderSecretChange(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(KafkaSecurityConfigs.SASL_JAAS_CONFIG, "staticLoginModule required;") + props.put(SaslConfigs.SASL_JAAS_CONFIG, "staticLoginModule required;") props.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, "config-encoder-secret") val config = KafkaConfig(props) config.dynamicConfig.initialize(None, None) val dynamicProps = new Properties - dynamicProps.put(KafkaSecurityConfigs.SASL_JAAS_CONFIG, "dynamicLoginModule required;") + dynamicProps.put(SaslConfigs.SASL_JAAS_CONFIG, "dynamicLoginModule required;") val persistedProps = config.dynamicConfig.toPersistentProps(dynamicProps, perBrokerConfig = true) - assertFalse(persistedProps.getProperty(KafkaSecurityConfigs.SASL_JAAS_CONFIG).contains("LoginModule"), + assertFalse(persistedProps.getProperty(SaslConfigs.SASL_JAAS_CONFIG).contains("LoginModule"), "Password not encoded") config.dynamicConfig.updateBrokerConfig(0, persistedProps) - assertEquals("dynamicLoginModule required;", config.values.get(KafkaSecurityConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value) + assertEquals("dynamicLoginModule required;", config.values.get(SaslConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value) // New config with same secret should use the dynamic password config val newConfigWithSameSecret = KafkaConfig(props) newConfigWithSameSecret.dynamicConfig.initialize(None, None) newConfigWithSameSecret.dynamicConfig.updateBrokerConfig(0, persistedProps) - assertEquals("dynamicLoginModule required;", newConfigWithSameSecret.values.get(KafkaSecurityConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value) + assertEquals("dynamicLoginModule required;", newConfigWithSameSecret.values.get(SaslConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value) // New config with new secret should use the dynamic password config if new and old secrets are configured in KafkaConfig props.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, "new-encoder-secret") props.put(PasswordEncoderConfigs.PASSWORD_ENCODER_OLD_SECRET_CONFIG, "config-encoder-secret") val newConfigWithNewAndOldSecret = KafkaConfig(props) newConfigWithNewAndOldSecret.dynamicConfig.updateBrokerConfig(0, persistedProps) - assertEquals("dynamicLoginModule required;", newConfigWithSameSecret.values.get(KafkaSecurityConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value) + assertEquals("dynamicLoginModule required;", newConfigWithSameSecret.values.get(SaslConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value) // New config with new secret alone should revert to static password config since dynamic config cannot be decoded props.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, "another-new-encoder-secret") val newConfigWithNewSecret = KafkaConfig(props) newConfigWithNewSecret.dynamicConfig.updateBrokerConfig(0, persistedProps) - assertEquals("staticLoginModule required;", newConfigWithNewSecret.values.get(KafkaSecurityConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value) + assertEquals("staticLoginModule required;", newConfigWithNewSecret.values.get(SaslConfigs.SASL_JAAS_CONFIG).asInstanceOf[Password].value) } @Test @@ -616,15 +616,15 @@ class DynamicBrokerConfigTest { when(zkClient.getEntityConfigs(anyString(), anyString())).thenReturn(new java.util.Properties()) val initialProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092) - initialProps.remove(KafkaConfig.BackgroundThreadsProp) + initialProps.remove(ServerConfigs.BACKGROUND_THREADS_CONFIG) val oldConfig = KafkaConfig.fromProps(initialProps) val dynamicBrokerConfig = new DynamicBrokerConfig(oldConfig) dynamicBrokerConfig.initialize(Some(zkClient), None) dynamicBrokerConfig.addBrokerReconfigurable(new TestDynamicThreadPool) val newprops = new Properties() - newprops.put(KafkaConfig.NumIoThreadsProp, "10") - newprops.put(KafkaConfig.BackgroundThreadsProp, "100") + newprops.put(ServerConfigs.NUM_IO_THREADS_CONFIG, "10") + newprops.put(ServerConfigs.BACKGROUND_THREADS_CONFIG, "100") dynamicBrokerConfig.updateBrokerConfig(0, newprops) } @@ -639,7 +639,7 @@ class DynamicBrokerConfigTest { var newProps = new Properties() newProps.put(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, "9999") - newProps.put(KafkaConfig.MessageMaxBytesProp, "2222") + newProps.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "2222") config.dynamicConfig.updateDefaultConfig(newProps) assertEquals(9999, config.maxConnections) @@ -647,7 +647,7 @@ class DynamicBrokerConfigTest { newProps = new Properties() newProps.put(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, "INVALID_INT") - newProps.put(KafkaConfig.MessageMaxBytesProp, "1111") + newProps.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "1111") config.dynamicConfig.updateDefaultConfig(newProps) // Invalid value should be skipped and reassigned as default value @@ -846,8 +846,8 @@ class TestDynamicThreadPool() extends BrokerReconfigurable { } override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { - assertEquals(Defaults.NUM_IO_THREADS, oldConfig.numIoThreads) - assertEquals(Defaults.BACKGROUND_THREADS, oldConfig.backgroundThreads) + assertEquals(ServerConfigs.NUM_IO_THREADS_DEFAULT, oldConfig.numIoThreads) + assertEquals(ServerConfigs.BACKGROUND_THREADS_DEFAULT, oldConfig.backgroundThreads) assertEquals(10, newConfig.numIoThreads) assertEquals(100, newConfig.backgroundThreads) diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala index 1c5d0a76f0f8a..6a7aa13fd4f4a 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala @@ -23,6 +23,7 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -74,7 +75,7 @@ class FetchRequestMaxBytesTest extends BaseRequestTest { override protected def brokerPropertyOverrides(properties: Properties): Unit = { super.brokerPropertyOverrides(properties) - properties.put(KafkaConfig.FetchMaxBytes, "1024") + properties.put(ServerConfigs.FETCH_MAX_BYTES_CONFIG, "1024") } private def createTopics(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala index a1773086ba61d..cc324a9182f10 100644 --- a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala @@ -21,7 +21,7 @@ import kafka.server.metadata.ZkMetadataCache import kafka.utils.TestUtils import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion} import org.apache.kafka.common.feature.{Features, SupportedVersionRange} -import org.apache.kafka.server.common.{Features => JFeatures} +import org.apache.kafka.server.common.{FinalizedFeatures => JFeatures} import org.apache.kafka.common.utils.Exit import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_3_2_IV0 diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index ce7acfe821f8d..bd0ded29debf9 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -79,7 +79,7 @@ import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.server.ClientMetricsManager import org.apache.kafka.server.authorizer.{Action, AuthorizationResult, Authorizer} import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1} -import org.apache.kafka.server.common.{Features, MetadataVersion} +import org.apache.kafka.server.common.{FinalizedFeatures, GroupVersion, MetadataVersion} import org.apache.kafka.server.config._ import org.apache.kafka.server.metrics.ClientMetricsTestUtils import org.apache.kafka.server.util.{FutureUtils, MockTime} @@ -199,7 +199,7 @@ class KafkaApisTest extends Logging { BrokerFeatures.defaultSupportedFeatures(true), true, false, - () => new Features(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, raftSupport)) + () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, raftSupport)) val clientMetricsManagerOpt = if (raftSupport) Some(clientMetricsManager) else None @@ -1180,7 +1180,7 @@ class KafkaApisTest extends Logging { val requestTimeout = 10 val topicConfigOverride = mutable.Map.empty[String, String] - topicConfigOverride.put(KafkaConfig.RequestTimeoutMsProp, requestTimeout.toString) + topicConfigOverride.put(ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG, requestTimeout.toString) val groupId = "group" val topicName = @@ -3155,7 +3155,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(0), ArgumentMatchers.eq(TransactionResult.COMMIT), - ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS)) + ArgumentMatchers.eq(Duration.ofMillis(ServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT)) )).thenReturn(CompletableFuture.completedFuture[Void](null)) when(groupCoordinator.completeTransaction( @@ -3164,7 +3164,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(0), ArgumentMatchers.eq(TransactionResult.ABORT), - ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS)) + ArgumentMatchers.eq(Duration.ofMillis(ServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT)) )).thenReturn(CompletableFuture.completedFuture[Void](null)) val entriesPerPartition: ArgumentCaptor[Map[TopicPartition, MemoryRecords]] = @@ -3173,7 +3173,7 @@ class KafkaApisTest extends Logging { ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords( - ArgumentMatchers.eq(Defaults.REQUEST_TIMEOUT_MS.toLong), + ArgumentMatchers.eq(ServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT.toLong), ArgumentMatchers.eq(-1), ArgumentMatchers.eq(true), ArgumentMatchers.eq(AppendOrigin.COORDINATOR), @@ -3274,7 +3274,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(0), ArgumentMatchers.eq(TransactionResult.COMMIT), - ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS)) + ArgumentMatchers.eq(Duration.ofMillis(ServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT)) )).thenReturn(FutureUtils.failedFuture[Void](error.exception())) kafkaApis = createKafkaApis(overrideProperties = Map( GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true" @@ -7020,6 +7020,16 @@ class KafkaApisTest extends Logging { @Test def testConsumerGroupHeartbeatRequest(): Unit = { + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.features()).thenReturn { + new FinalizedFeatures( + MetadataVersion.latestTesting(), + Collections.singletonMap(GroupVersion.FEATURE_NAME, GroupVersion.GV_1.featureLevel()), + 0, + true + ) + } + val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequestData().setGroupId("group") val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest, true).build()) @@ -7029,9 +7039,10 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, consumerGroupHeartbeatRequest )).thenReturn(future) - kafkaApis = createKafkaApis(overrideProperties = Map( - GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true" - )) + kafkaApis = createKafkaApis( + overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,consumer"), + raftSupport = true + ) kafkaApis.handle(requestChannelRequest, RequestLocal.NoCaching) val consumerGroupHeartbeatResponse = new ConsumerGroupHeartbeatResponseData() @@ -7044,6 +7055,16 @@ class KafkaApisTest extends Logging { @Test def testConsumerGroupHeartbeatRequestFutureFailed(): Unit = { + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.features()).thenReturn { + new FinalizedFeatures( + MetadataVersion.latestTesting(), + Collections.singletonMap(GroupVersion.FEATURE_NAME, GroupVersion.GV_1.featureLevel()), + 0, + true + ) + } + val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequestData().setGroupId("group") val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest, true).build()) @@ -7053,9 +7074,10 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, consumerGroupHeartbeatRequest )).thenReturn(future) - kafkaApis = createKafkaApis(overrideProperties = Map( - GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true" - )) + kafkaApis = createKafkaApis( + overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,consumer"), + raftSupport = true + ) kafkaApis.handle(requestChannelRequest, RequestLocal.NoCaching) future.completeExceptionally(Errors.FENCED_MEMBER_EPOCH.exception) @@ -7065,6 +7087,16 @@ class KafkaApisTest extends Logging { @Test def testConsumerGroupHeartbeatRequestAuthorizationFailed(): Unit = { + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.features()).thenReturn { + new FinalizedFeatures( + MetadataVersion.latestTesting(), + Collections.singletonMap(GroupVersion.FEATURE_NAME, GroupVersion.GV_1.featureLevel()), + 0, + true + ) + } + val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequestData().setGroupId("group") val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest, true).build()) @@ -7074,7 +7106,8 @@ class KafkaApisTest extends Logging { .thenReturn(Seq(AuthorizationResult.DENIED).asJava) kafkaApis = createKafkaApis( authorizer = Some(authorizer), - overrideProperties = Map(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true") + overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,consumer"), + raftSupport = true ) kafkaApis.handle(requestChannelRequest, RequestLocal.NoCaching) @@ -7084,6 +7117,16 @@ class KafkaApisTest extends Logging { @Test def testConsumerGroupDescribe(): Unit = { + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.features()).thenReturn { + new FinalizedFeatures( + MetadataVersion.latestTesting(), + Collections.singletonMap(GroupVersion.FEATURE_NAME, GroupVersion.GV_1.featureLevel()), + 0, + true + ) + } + val groupIds = List("group-id-0", "group-id-1", "group-id-2").asJava val consumerGroupDescribeRequestData = new ConsumerGroupDescribeRequestData() consumerGroupDescribeRequestData.groupIds.addAll(groupIds) @@ -7095,7 +7138,8 @@ class KafkaApisTest extends Logging { any[util.List[String]] )).thenReturn(future) kafkaApis = createKafkaApis( - overrideProperties = Map(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true") + overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,consumer"), + raftSupport = true ) kafkaApis.handle(requestChannelRequest, RequestLocal.NoCaching) @@ -7134,6 +7178,16 @@ class KafkaApisTest extends Logging { @Test def testConsumerGroupDescribeAuthorizationFailed(): Unit = { + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.features()).thenReturn { + new FinalizedFeatures( + MetadataVersion.latestTesting(), + Collections.singletonMap(GroupVersion.FEATURE_NAME, GroupVersion.GV_1.featureLevel()), + 0, + true + ) + } + val consumerGroupDescribeRequestData = new ConsumerGroupDescribeRequestData() consumerGroupDescribeRequestData.groupIds.add("group-id") val requestChannelRequest = buildRequest(new ConsumerGroupDescribeRequest.Builder(consumerGroupDescribeRequestData, true).build()) @@ -7150,7 +7204,8 @@ class KafkaApisTest extends Logging { future.complete(List().asJava) kafkaApis = createKafkaApis( authorizer = Some(authorizer), - overrideProperties = Map(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true") + overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,consumer"), + raftSupport = true ) kafkaApis.handle(requestChannelRequest, RequestLocal.NoCaching) @@ -7160,6 +7215,16 @@ class KafkaApisTest extends Logging { @Test def testConsumerGroupDescribeFutureFailed(): Unit = { + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.features()).thenReturn { + new FinalizedFeatures( + MetadataVersion.latestTesting(), + Collections.singletonMap(GroupVersion.FEATURE_NAME, GroupVersion.GV_1.featureLevel()), + 0, + true + ) + } + val consumerGroupDescribeRequestData = new ConsumerGroupDescribeRequestData() consumerGroupDescribeRequestData.groupIds.add("group-id") val requestChannelRequest = buildRequest(new ConsumerGroupDescribeRequest.Builder(consumerGroupDescribeRequestData, true).build()) @@ -7169,9 +7234,10 @@ class KafkaApisTest extends Logging { any[RequestContext], any[util.List[String]] )).thenReturn(future) - kafkaApis = createKafkaApis(overrideProperties = Map( - GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true" - )) + kafkaApis = createKafkaApis( + overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,consumer"), + raftSupport = true + ) kafkaApis.handle(requestChannelRequest, RequestLocal.NoCaching) future.completeExceptionally(Errors.FENCED_MEMBER_EPOCH.exception) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 8513854be7b4f..933f514df595c 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -25,12 +25,13 @@ import kafka.security.authorizer.AclAuthorizer import kafka.utils.TestUtils.assertBadConfigContainingMessage import kafka.utils.{CoreUtils, TestUtils} import org.apache.kafka.common.Node -import org.apache.kafka.common.config.{ConfigException, TopicConfig} +import org.apache.kafka.common.config.{ConfigException, SaslConfigs, SecurityConfig, SslConfigs, TopicConfig} import org.apache.kafka.common.metrics.Sensor import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.record.{CompressionType, Records} import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.compress.{GzipCompression, Lz4Compression, ZstdCompression} +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.coordinator.group.ConsumerGroupMigrationPolicy import org.apache.kafka.coordinator.group.Group.GroupType import org.apache.kafka.coordinator.group.GroupCoordinatorConfig @@ -40,13 +41,14 @@ import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.security.PasswordEncoderConfigs import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_8_2, IBP_3_0_IV1} -import org.apache.kafka.server.config.{KRaftConfigs, KafkaSecurityConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs} +import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, QuotaConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.MetricConfigs import org.apache.kafka.storage.internals.log.CleanerConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.junit.jupiter.api.function.Executable + import scala.annotation.nowarn import scala.jdk.CollectionConverters._ @@ -158,7 +160,7 @@ class KafkaConfigTest { val port = 9999 val hostName = "fake-host" val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"PLAINTEXT://$hostName:$port") val serverConfig = KafkaConfig.fromProps(props) @@ -189,7 +191,7 @@ class KafkaConfigTest { @Test def testDuplicateListeners(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") // listeners with duplicate port @@ -215,7 +217,7 @@ class KafkaConfigTest { @Test def testIPv4AndIPv6SamePortListeners(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") + props.put(ServerConfigs.BROKER_ID_CONFIG, "1") props.put(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://[::1]:9092,SSL://[::1]:9092") @@ -455,7 +457,7 @@ class KafkaConfigTest { @Test def testControllerListenerNameDoesNotMapToPlaintextByDefaultForNonKRaft(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://localhost:9092") assertBadConfigContainingMessage(props, @@ -468,7 +470,7 @@ class KafkaConfigTest { @Test def testBadListenerProtocol(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "BAD://localhost:9091") @@ -478,7 +480,7 @@ class KafkaConfigTest { @Test def testListenerNamesWithAdvertisedListenerUnset(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CLIENT://localhost:9091,REPLICATION://localhost:9092,INTERNAL://localhost:9093") @@ -502,7 +504,7 @@ class KafkaConfigTest { @Test def testListenerAndAdvertisedListenerNames(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "EXTERNAL://localhost:9091,INTERNAL://localhost:9093") @@ -533,7 +535,7 @@ class KafkaConfigTest { @Test def testListenerNameMissingFromListenerSecurityProtocolMap(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9091,REPLICATION://localhost:9092") @@ -544,7 +546,7 @@ class KafkaConfigTest { @Test def testInterBrokerListenerNameMissingFromListenerSecurityProtocolMap(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9091") @@ -555,7 +557,7 @@ class KafkaConfigTest { @Test def testInterBrokerListenerNameAndSecurityProtocolSet(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9091") @@ -567,7 +569,7 @@ class KafkaConfigTest { @Test def testCaseInsensitiveListenerProtocol(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "plaintext://localhost:9091,SsL://localhost:9092") val config = KafkaConfig.fromProps(props) @@ -582,7 +584,7 @@ class KafkaConfigTest { @Test def testListenerDefaults(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") // configuration with no listeners @@ -596,7 +598,7 @@ class KafkaConfigTest { @Test def testVersionConfiguration(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") val conf = KafkaConfig.fromProps(props) assertEquals(MetadataVersion.latestProduction, conf.interBrokerProtocolVersion) @@ -706,31 +708,31 @@ class KafkaConfigTest { @Test def testInvalidCompressionType(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.setProperty(KafkaConfig.CompressionTypeProp, "abc") + props.setProperty(ServerConfigs.COMPRESSION_TYPE_CONFIG, "abc") assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @Test def testInvalidGzipCompressionLevel(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.setProperty(KafkaConfig.CompressionTypeProp, "gzip") - props.setProperty(KafkaConfig.CompressionGzipLevelProp, (GzipCompression.MAX_LEVEL + 1).toString) + props.setProperty(ServerConfigs.COMPRESSION_TYPE_CONFIG, "gzip") + props.setProperty(ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG, (GzipCompression.MAX_LEVEL + 1).toString) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @Test def testInvalidLz4CompressionLevel(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.setProperty(KafkaConfig.CompressionTypeProp, "lz4") - props.setProperty(KafkaConfig.CompressionLz4LevelProp, (Lz4Compression.MAX_LEVEL + 1).toString) + props.setProperty(ServerConfigs.COMPRESSION_TYPE_CONFIG, "lz4") + props.setProperty(ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG, (Lz4Compression.MAX_LEVEL + 1).toString) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @Test def testInvalidZstdCompressionLevel(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.setProperty(KafkaConfig.CompressionTypeProp, "zstd") - props.setProperty(KafkaConfig.CompressionZstdLevelProp, (ZstdCompression.MAX_LEVEL + 1).toString) + props.setProperty(ServerConfigs.COMPRESSION_TYPE_CONFIG, "zstd") + props.setProperty(ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG, (ZstdCompression.MAX_LEVEL + 1).toString) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @@ -820,16 +822,16 @@ class KafkaConfigTest { case ZkConfigs.ZK_SSL_CRL_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean") case ZkConfigs.ZK_SSL_OCSP_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean") - case KafkaConfig.BrokerIdProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.NumNetworkThreadsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.NumIoThreadsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.BackgroundThreadsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.QueuedMaxRequestsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.NumReplicaAlterLogDirsThreadsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.QueuedMaxBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.RequestTimeoutMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.ConnectionSetupTimeoutMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.ConnectionSetupTimeoutMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.BROKER_ID_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.NUM_NETWORK_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case ServerConfigs.NUM_IO_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case ServerConfigs.BACKGROUND_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case ServerConfigs.QUEUED_MAX_REQUESTS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case ServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.QUEUED_MAX_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") // KRaft mode configs case KRaftConfigs.PROCESS_ROLES_CONFIG => // ignore @@ -845,7 +847,7 @@ class KafkaConfigTest { case KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG => // ignore string case KRaftConfigs.METADATA_MAX_IDLE_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.AuthorizerClassNameProp => //ignore string + case ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG => //ignore string case ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG => //ignore string case SocketServerConfigs.SOCKET_SEND_BUFFER_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") @@ -909,9 +911,9 @@ class KafkaConfigTest { case ReplicationConfigs.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case ReplicationConfigs.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") - case KafkaConfig.ControlledShutdownMaxRetriesProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.ControlledShutdownRetryBackoffMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.ControlledShutdownEnableProp => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") + case ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") case GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case GroupCoordinatorConfig.GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") @@ -934,78 +936,78 @@ class KafkaConfigTest { case TransactionLogConfigs.TRANSACTIONS_TOPIC_SEGMENT_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0", "-2") case TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0", "-2") case QuotaConfigs.QUOTA_WINDOW_SIZE_SECONDS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") + case ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") case MetricConfigs.METRIC_NUM_SAMPLES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1", "0") case MetricConfigs.METRIC_SAMPLE_WINDOW_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1", "0") case MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG => // ignore string case MetricConfigs.METRIC_RECORDING_LEVEL_CONFIG => // ignore string + case ServerConfigs.BROKER_RACK_CONFIG => // ignore string - case KafkaConfig.CompressionGzipLevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.CompressionLz4LevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.CompressionZstdLevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", ZstdCompression.MAX_LEVEL + 1) + case ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", ZstdCompression.MAX_LEVEL + 1) - case KafkaConfig.RackProp => // ignore string //SSL Configs - case KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG => - case KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG => - case KafkaSecurityConfigs.SSL_PROTOCOL_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_PROVIDER_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_ENABLED_PROTOCOLS_CONFIG => - case KafkaSecurityConfigs.SSL_KEYSTORE_TYPE_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_KEYSTORE_LOCATION_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_KEYSTORE_KEY_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_TRUSTSTORE_TYPE_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG => - case KafkaSecurityConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG => - case KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_CIPHER_SUITES_CONFIG => // ignore string - case KafkaSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_CONFIG => // ignore string + case BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG => + case BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG => + case SslConfigs.SSL_PROTOCOL_CONFIG => // ignore string + case SslConfigs.SSL_PROVIDER_CONFIG => // ignore string + case SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG => + case SslConfigs.SSL_KEYSTORE_TYPE_CONFIG => // ignore string + case SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG => // ignore string + case SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG => // ignore string + case SslConfigs.SSL_KEY_PASSWORD_CONFIG => // ignore string + case SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG => // ignore string + case SslConfigs.SSL_KEYSTORE_KEY_CONFIG => // ignore string + case SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG => // ignore string + case SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG => // ignore string + case SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG => // ignore string + case SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG => // ignore string + case SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG => + case SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG => + case BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG => // ignore string + case SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG => // ignore string + case SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG => // ignore string + case SslConfigs.SSL_CIPHER_SUITES_CONFIG => // ignore string + case BrokerSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_CONFIG => // ignore string //Sasl Configs case KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG => // ignore - case KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG => // ignore - case KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG => - case KafkaSecurityConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_CONFIG => - case KafkaSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG => - case KafkaSecurityConfigs.SASL_LOGIN_CLASS_CONFIG => - case KafkaSecurityConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS_CONFIG => - case KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG => // ignore string - case KafkaSecurityConfigs.SASL_KERBEROS_KINIT_CMD_CONFIG => - case KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_CONFIG => - case KafkaSecurityConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_CONFIG => - case KafkaSecurityConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_CONFIG => - case KafkaSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG => // ignore string - case KafkaSecurityConfigs.SASL_JAAS_CONFIG => - case KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_CONFIG => - case KafkaSecurityConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_CONFIG => - case KafkaSecurityConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_CONFIG => - case KafkaSecurityConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_CONFIG => - case KafkaSecurityConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_CONFIG => - case KafkaSecurityConfigs.SASL_LOGIN_READ_TIMEOUT_MS_CONFIG => - case KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_CONFIG => - case KafkaSecurityConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_CONFIG => - case KafkaSecurityConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_CONFIG => - case KafkaSecurityConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_CONFIG => - case KafkaSecurityConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_CONFIG => - case KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_CONFIG => - case KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_CONFIG => - case KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_CONFIG => - case KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_CONFIG => - case KafkaSecurityConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_CONFIG => - case KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_CONFIG => - case KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_CONFIG => + case BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG => // ignore + case BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG => + case SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS => + case BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG => + case SaslConfigs.SASL_LOGIN_CLASS => + case SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS => + case SaslConfigs.SASL_KERBEROS_SERVICE_NAME => // ignore string + case SaslConfigs.SASL_KERBEROS_KINIT_CMD => + case SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR => + case SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER => + case SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN => + case BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG => // ignore string + case SaslConfigs.SASL_JAAS_CONFIG => + case SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR => + case SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER => + case SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS => + case SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS => + case SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS => + case SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS => + case SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS => + case SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS => + case SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME => + case SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME => + case SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL => + case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL => + case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS => + case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS => + case SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS => + case SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS => + case SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE => + case SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER => // Security config - case KafkaSecurityConfigs.SECURITY_PROVIDER_CLASS_CONFIG => + case SecurityConfig.SECURITY_PROVIDERS_CONFIG => // Password encoder configs case PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG => @@ -1016,17 +1018,17 @@ class KafkaConfigTest { case PasswordEncoderConfigs.PASSWORD_ENCODER_ITERATIONS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1", "0") //delegation token configs - case KafkaConfig.DelegationTokenSecretKeyAliasProp => // ignore - case KafkaConfig.DelegationTokenSecretKeyProp => // ignore - case KafkaConfig.DelegationTokenMaxLifeTimeProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.DelegationTokenExpiryTimeMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.DelegationTokenExpiryCheckIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG => // ignore + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG => // ignore + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFETIME_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") //Kafka Yammer metrics reporter configs case MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG => // ignore case MetricConfigs.KAFKA_METRICS_POLLING_INTERVAL_SECONDS_CONFIG => //ignore - case KafkaSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") // Raft Quorum Configs case QuorumConfig.QUORUM_VOTERS_CONFIG => // ignore string @@ -1180,9 +1182,9 @@ class KafkaConfigTest { defaults.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "127.0.0.1:2181") // For ZkConnectionTimeoutMs defaults.setProperty(ZkConfigs.ZK_SESSION_TIMEOUT_MS_CONFIG, "1234") - defaults.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") - defaults.setProperty(KafkaConfig.MaxReservedBrokerIdProp, "1") - defaults.setProperty(KafkaConfig.BrokerIdProp, "1") + defaults.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") + defaults.setProperty(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG, "1") + defaults.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") defaults.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:1122") defaults.setProperty(SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG, "127.0.0.1:2, 127.0.0.2:3") defaults.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, "/tmp1,/tmp2") @@ -1215,7 +1217,7 @@ class KafkaConfigTest { assertEquals(24 * 60L * 60L * 1000L, config.delegationTokenExpiryTimeMs) assertEquals(1 * 60L * 1000L * 60, config.delegationTokenExpiryCheckIntervalMs) - defaults.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "1234567890") + defaults.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "1234567890") val config1 = KafkaConfig.fromProps(defaults) assertEquals(true, config1.tokenAuthEnabled) } @@ -1436,7 +1438,7 @@ class KafkaConfigTest { // -1 is the default for both node.id and broker.id val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") assertFalse(isValidKafkaConfig(props)) } @@ -1473,7 +1475,7 @@ class KafkaConfigTest { props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners) props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) props.setProperty(KRaftConfigs.NODE_ID_CONFIG, negativeTwoNodeId.toString) - props.setProperty(KafkaConfig.BrokerIdProp, negativeTwoNodeId.toString) + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, negativeTwoNodeId.toString) assertFalse(isValidKafkaConfig(props)) } @@ -1486,7 +1488,7 @@ class KafkaConfigTest { val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners) props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") KafkaConfig.fromProps(props) } @@ -1496,7 +1498,7 @@ class KafkaConfigTest { val props = TestUtils.createBrokerConfig(-1, TestUtils.MockZkConnect, port = TestUtils.MockZkPort) val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners) - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") assertFalse(isValidKafkaConfig(props)) } @@ -1564,9 +1566,9 @@ class KafkaConfigTest { @Test def testPopulateSynonymsOnMapWithoutNodeId(): Unit = { val input = new util.HashMap[String, String]() - input.put(KafkaConfig.BrokerIdProp, "4") + input.put(ServerConfigs.BROKER_ID_CONFIG, "4") val expectedOutput = new util.HashMap[String, String]() - expectedOutput.put(KafkaConfig.BrokerIdProp, "4") + expectedOutput.put(ServerConfigs.BROKER_ID_CONFIG, "4") expectedOutput.put(KRaftConfigs.NODE_ID_CONFIG, "4") assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input)) } @@ -1576,7 +1578,7 @@ class KafkaConfigTest { val input = new util.HashMap[String, String]() input.put(KRaftConfigs.NODE_ID_CONFIG, "4") val expectedOutput = new util.HashMap[String, String]() - expectedOutput.put(KafkaConfig.BrokerIdProp, "4") + expectedOutput.put(ServerConfigs.BROKER_ID_CONFIG, "4") expectedOutput.put(KRaftConfigs.NODE_ID_CONFIG, "4") assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input)) } @@ -1584,7 +1586,7 @@ class KafkaConfigTest { @Test def testNodeIdMustNotBeDifferentThanBrokerId(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2") assertEquals("You must set `node.id` to the same value as `broker.id`.", assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage()) @@ -1605,13 +1607,13 @@ class KafkaConfigTest { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") - props.setProperty(KafkaConfig.BrokerIdProp, "3") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "3") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") val config = KafkaConfig.fromProps(props) assertEquals(3, config.brokerId) assertEquals(3, config.nodeId) val originals = config.originals() - assertEquals("3", originals.get(KafkaConfig.BrokerIdProp)) + assertEquals("3", originals.get(ServerConfigs.BROKER_ID_CONFIG)) assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG)) } @@ -1632,7 +1634,7 @@ class KafkaConfigTest { assertEquals(3, config.brokerId) assertEquals(3, config.nodeId) val originals = config.originals() - assertEquals("3", originals.get(KafkaConfig.BrokerIdProp)) + assertEquals("3", originals.get(ServerConfigs.BROKER_ID_CONFIG)) assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG)) } @@ -1641,17 +1643,17 @@ class KafkaConfigTest { val props = new Properties() props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") val config = KafkaConfig.fromProps(props) - assertNotNull(config.getLong(KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_CONFIG)) - assertNotNull(config.getLong(KafkaSecurityConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_CONFIG)) + assertNotNull(config.getLong(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS)) + assertNotNull(config.getLong(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS)) } @Test def testInvalidAuthorizerClassName(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val configs = new util.HashMap[Object, Object](props) - configs.put(KafkaConfig.AuthorizerClassNameProp, null) + configs.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, null) val ce = assertThrows(classOf[ConfigException], () => KafkaConfig.apply(configs)) - assertTrue(ce.getMessage.contains(KafkaConfig.AuthorizerClassNameProp)) + assertTrue(ce.getMessage.contains(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG)) } @Test @@ -1678,7 +1680,7 @@ class KafkaConfigTest { def testEarlyStartListeners(): Unit = { val props = new Properties() props.putAll(kraftProps()) - props.setProperty(KafkaConfig.EarlyStartListenersProp, "INTERNAL,INTERNAL2") + props.setProperty(ServerConfigs.EARLY_START_LISTENERS_CONFIG, "INTERNAL,INTERNAL2") props.setProperty(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, "INTERNAL") props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "INTERNAL:PLAINTEXT,INTERNAL2:PLAINTEXT,CONTROLLER:PLAINTEXT") @@ -1693,7 +1695,7 @@ class KafkaConfigTest { def testEarlyStartListenersMustBeListeners(): Unit = { val props = new Properties() props.putAll(kraftProps()) - props.setProperty(KafkaConfig.EarlyStartListenersProp, "INTERNAL") + props.setProperty(ServerConfigs.EARLY_START_LISTENERS_CONFIG, "INTERNAL") assertEquals("early.start.listeners contains listener INTERNAL, but this is not " + "contained in listeners or controller.listener.names", assertThrows(classOf[ConfigException], () => new KafkaConfig(props)).getMessage) @@ -1765,7 +1767,7 @@ class KafkaConfigTest { KafkaConfig.fromProps(props) // Check that we allow authorizer to be set - props.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getCanonicalName) + props.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getCanonicalName) KafkaConfig.fromProps(props) // Don't allow migration startup with an older IBP @@ -1852,6 +1854,12 @@ class KafkaConfigTest { @Test def testGroupCoordinatorRebalanceProtocols(): Unit = { val props = new Properties() + + // consumer cannot be enabled in ZK mode. + props.put(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer") + assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) + + // Setting KRaft's properties. props.putAll(kraftProps()) // Only classic and consumer are supported. @@ -1892,21 +1900,14 @@ class KafkaConfigTest { } } - @Test - def testMultipleLogDirectoriesNotSupportedWithRemoteLogStorage(): Unit = { - val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, String.valueOf(true)) - props.put(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/a,/tmp/b") - - val caught = assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) - assertTrue(caught.getMessage.contains("Multiple log directories `/tmp/a,/tmp/b` are not supported when remote log storage is enabled")) - } - @Test def testSingleLogDirectoryWithRemoteLogStorage(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, String.valueOf(true)) props.put(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/a") assertDoesNotThrow(() => KafkaConfig.fromProps(props)) + + props.put(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/a,/tmp/b") + assertDoesNotThrow(() => KafkaConfig.fromProps(props)) } } diff --git a/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala b/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala index a27a49f9b2aa9..9cc0204210712 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala @@ -21,6 +21,7 @@ import kafka.metrics.KafkaMetricsReporter import kafka.utils.{CoreUtils, TestUtils, VerifiableProperties} import kafka.server.QuorumTestHarness import org.apache.kafka.common.{ClusterResource, ClusterResourceListener} +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.metrics.MetricConfigs import org.apache.kafka.test.MockMetricsReporter import org.junit.jupiter.api.Assertions._ @@ -61,7 +62,7 @@ object KafkaMetricReporterClusterIdTest { // Because this code is run during the test setUp phase, if we throw an exception here, // it just results in the test itself being declared "not found" rather than failing. // So we track an error message which we will check later in the test body. - val brokerId = configs.get(KafkaConfig.BrokerIdProp) + val brokerId = configs.get(ServerConfigs.BROKER_ID_CONFIG) if (brokerId == null) setupError.compareAndSet("", "No value was set for the broker id.") else if (!brokerId.isInstanceOf[String]) @@ -85,8 +86,8 @@ class KafkaMetricReporterClusterIdTest extends QuorumTestHarness { val props = TestUtils.createBrokerConfig(1, zkConnect) props.setProperty(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG, "kafka.server.KafkaMetricReporterClusterIdTest$MockKafkaMetricsReporter") props.setProperty(MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG, "kafka.server.KafkaMetricReporterClusterIdTest$MockBrokerMetricsReporter") - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "true") - props.setProperty(KafkaConfig.BrokerIdProp, "-1") + props.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "true") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "-1") config = KafkaConfig.fromProps(props) server = new KafkaServer(config, threadNamePrefix = Option(this.getClass.getName)) server.startup() diff --git a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala index 9886b1e056f4a..09f14fdcaa7a2 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala @@ -20,6 +20,7 @@ import java.util import java.util.concurrent.atomic.AtomicReference import kafka.utils.{CoreUtils, TestUtils} import org.apache.kafka.common.metrics.{KafkaMetric, MetricsContext, MetricsReporter} +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.metrics.MetricConfigs import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.Assertions._ @@ -72,8 +73,8 @@ class KafkaMetricsReporterTest extends QuorumTestHarness { super.setUp(testInfo) val props = TestUtils.createBrokerConfig(1, zkConnectOrNull) props.setProperty(MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG, "kafka.server.KafkaMetricsReporterTest$MockMetricsReporter") - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "true") - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "true") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") config = KafkaConfig.fromProps(props) broker = createBroker(config, threadNamePrefix = Option(this.getClass.getName)) broker.startup() diff --git a/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala index b99239497e86c..3fcffa893aff7 100644 --- a/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala @@ -17,13 +17,14 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions import org.apache.kafka.common.message.ListGroupsResponseData import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.apache.kafka.coordinator.group.Group +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.{assertEquals, fail} import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.extension.ExtendWith @@ -33,13 +34,18 @@ import org.junit.jupiter.api.extension.ExtendWith @ClusterTestDefaults(types = Array(Type.KRAFT)) @Tag("integration") class ListGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testListGroupsWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testListGroups(true) } diff --git a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala index 53bd2b0e0da4b..a8b7a280defeb 100644 --- a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala @@ -28,7 +28,8 @@ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{KafkaStorageException, NotLeaderOrFollowerException} import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.metadata.BrokerState +import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.params.provider.ValueSource @@ -52,6 +53,8 @@ class LogDirFailureTest extends IntegrationTestHarness { this.serverConfig.setProperty(ReplicationConfigs.REPLICA_HIGH_WATERMARK_CHECKPOINT_INTERVAL_MS_CONFIG, "60000") this.serverConfig.setProperty(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "1") + this.serverConfig.setProperty(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, "5000") + this.serverConfig.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") @BeforeEach override def setUp(testInfo: TestInfo): Unit = { @@ -66,6 +69,31 @@ class LogDirFailureTest extends IntegrationTestHarness { testProduceErrorsFromLogDirFailureOnLeader(Roll) } + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testLogDirNotificationTimeout(quorum: String): Unit = { + // Disable retries to allow exception to bubble up for validation + this.producerConfig.setProperty(ProducerConfig.RETRIES_CONFIG, "0") + this.producerConfig.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false") + val producer = createProducer() + + val partition = new TopicPartition(topic, 0) + + val leaderServerId = producer.partitionsFor(topic).asScala.find(_.partition() == 0).get.leader().id() + val leaderServer = brokers.find(_.config.brokerId == leaderServerId).get + + // shut down the controller to simulate the case where the broker is not able to send the log dir notification + controllerServer.shutdown() + controllerServer.awaitShutdown() + + TestUtils.causeLogDirFailure(Checkpoint, leaderServer, partition) + + TestUtils.waitUntilTrue(() => leaderServer.brokerState == BrokerState.SHUTTING_DOWN, + s"Expected broker to be in NOT_RUNNING state but was ${leaderServer.brokerState}", 15000) + // wait for actual shutdown (by default max 5 minutes for graceful shutdown) + leaderServer.awaitShutdown() + } + @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testIOExceptionDuringLogRoll(quorum: String): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/MockTierStateMachine.scala b/core/src/test/scala/unit/kafka/server/MockTierStateMachine.scala index 9e225607325d5..86df92d77daf9 100644 --- a/core/src/test/scala/unit/kafka/server/MockTierStateMachine.scala +++ b/core/src/test/scala/unit/kafka/server/MockTierStateMachine.scala @@ -20,9 +20,7 @@ package kafka.server import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.message.FetchResponseData -import java.util.Optional - -class MockTierStateMachine(leader: LeaderEndPoint) extends ReplicaFetcherTierStateMachine(leader, null) { +class MockTierStateMachine(leader: LeaderEndPoint) extends TierStateMachine(leader, null, false) { var fetcher: MockFetcherThread = _ @@ -37,11 +35,6 @@ class MockTierStateMachine(leader: LeaderEndPoint) extends ReplicaFetcherTierSta Fetching, Some(currentFetchState.currentLeaderEpoch)) } - override def maybeAdvanceState(topicPartition: TopicPartition, - currentFetchState: PartitionFetchState): Optional[PartitionFetchState] = { - Optional.of(currentFetchState) - } - def setFetcher(mockFetcherThread: MockFetcherThread): Unit = { fetcher = mockFetcherThread } diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala index 7791c6c942086..696e6534bf047 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala @@ -17,9 +17,10 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.fail import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.extension.ExtendWith @@ -30,22 +31,29 @@ import org.junit.jupiter.api.extension.ExtendWith @Tag("integration") class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testOffsetCommitWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testOffsetCommit(true) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ) def testOffsetCommitWithOldConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testOffsetCommit(false) } diff --git a/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala index 8fabac4bda77d..22c1e28548b96 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala @@ -17,9 +17,10 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.fail import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.extension.ExtendWith @@ -29,13 +30,18 @@ import org.junit.jupiter.api.extension.ExtendWith @ClusterTestDefaults(types = Array(Type.KRAFT)) @Tag("integration") class OffsetDeleteRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testOffsetDeleteWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testOffsetDelete(true) } diff --git a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala index 5f51d6cdb15b1..2e640bfea6f99 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala @@ -18,18 +18,16 @@ package kafka.server import kafka.test.ClusterInstance -import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import kafka.test.annotation.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.test.junit.ClusterTestExtensions - import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.message.OffsetFetchResponseData import org.apache.kafka.common.protocol.{ApiKeys, Errors} - +import org.apache.kafka.server.common.Features import org.junit.jupiter.api.Assertions.{assertEquals, fail} import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.extension.ExtendWith - import scala.jdk.CollectionConverters._ @Timeout(120) @@ -38,24 +36,31 @@ import scala.jdk.CollectionConverters._ @Tag("integration") class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testSingleGroupOffsetFetchWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testSingleGroupOffsetFetch(useNewProtocol = true, requireStable = true) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ) def testSingleGroupOffsetFetchWithOldConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testSingleGroupOffsetFetch(useNewProtocol = false, requireStable = false) } @@ -71,13 +76,18 @@ class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorB testSingleGroupOffsetFetch(useNewProtocol = false, requireStable = true) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testSingleGroupAllOffsetFetchWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testSingleGroupAllOffsetFetch(useNewProtocol = true, requireStable = true) } @@ -104,13 +114,18 @@ class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorB testSingleGroupAllOffsetFetch(useNewProtocol = false, requireStable = true) } - @ClusterTest(serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), - new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - )) + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"), + new ClusterConfigProperty(key = "group.consumer.max.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "600000"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + features = Array( + new ClusterFeature(feature = Features.GROUP_VERSION, version = 1) + ) + ) def testMultiGroupsOffsetFetchWithNewConsumerGroupProtocolAndNewGroupCoordinator(): Unit = { testMultipleGroupsOffsetFetch(useNewProtocol = true, requireStable = true) } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index d1382182ec79f..1f276a1035a5b 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -315,6 +315,81 @@ class ReplicaManagerTest { } } + @ParameterizedTest(name = "testMaybeAddLogDirFetchersPausingCleaning with futureLogCreated: {0}") + @ValueSource(booleans = Array(true, false)) + def testMaybeAddLogDirFetchersPausingCleaning(futureLogCreated: Boolean): Unit = { + val dir1 = TestUtils.tempDir() + val dir2 = TestUtils.tempDir() + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) + props.put("log.dirs", dir1.getAbsolutePath + "," + dir2.getAbsolutePath) + val config = KafkaConfig.fromProps(props) + val logManager = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(new Properties())) + val spyLogManager = spy(logManager) + val metadataCache: MetadataCache = mock(classOf[MetadataCache]) + mockGetAliveBrokerFunctions(metadataCache, Seq(new Node(0, "host0", 0))) + when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) + val tp0 = new TopicPartition(topic, 0) + val uuid = Uuid.randomUuid() + val rm = new ReplicaManager( + metrics = metrics, + config = config, + time = time, + scheduler = new MockScheduler(time), + logManager = spyLogManager, + quotaManagers = quotaManager, + metadataCache = metadataCache, + logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), + alterPartitionManager = alterPartitionManager) + + try { + val partition = rm.createPartition(tp0) + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, + new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints), Option.apply(uuid)) + + val response = rm.becomeLeaderOrFollower(0, new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(0) + .setIsr(Seq[Integer](0).asJava) + .setPartitionEpoch(0) + .setReplicas(Seq[Integer](0).asJava) + .setIsNew(false)).asJava, + Collections.singletonMap(topic, uuid), + Set(new Node(0, "host1", 0)).asJava).build(), (_, _) => ()) + // expect the errorCounts only has 1 entry with Errors.NONE + val errorCounts = response.errorCounts() + assertEquals(1, response.errorCounts().size()) + assertNotNull(errorCounts.get(Errors.NONE)) + spyLogManager.maybeUpdatePreferredLogDir(tp0, dir2.getAbsolutePath) + + if (futureLogCreated) { + // create future log before maybeAddLogDirFetchers invoked + partition.createLogIfNotExists(isNew = false, isFutureReplica = true, + new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints), None) + } else { + val mockLog = mock(classOf[UnifiedLog]) + when(spyLogManager.getLog(tp0, isFuture = true)).thenReturn(Option.apply(mockLog)) + when(mockLog.topicId).thenReturn(Option.apply(uuid)) + when(mockLog.parentDir).thenReturn(dir2.getAbsolutePath) + } + + val topicIdMap: Map[String, Option[Uuid]] = Map(topic -> Option.apply(uuid)) + rm.maybeAddLogDirFetchers(Set(partition), new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints), topicIdMap) + if (futureLogCreated) { + // since the futureLog is already created, we don't have to abort and pause the cleaning + verify(spyLogManager, never).abortAndPauseCleaning(any[TopicPartition]) + } else { + verify(spyLogManager, times(1)).abortAndPauseCleaning(any[TopicPartition]) + } + rm.replicaAlterLogDirsManager.fetcherThreadMap.values.foreach(t => t.fetchState(new TopicPartition(topic, 0)).foreach(s => assertEquals(0L, s.fetchOffset))) + } finally { + rm.shutdown(checkpointHW = false) + } + } + @Test def testClearPurgatoryOnBecomingFollower(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) @@ -3287,12 +3362,8 @@ class ReplicaManagerTest { val props = TestUtils.createBrokerConfig(brokerId, TestUtils.MockZkConnect) val path1 = TestUtils.tempRelativeDir("data").getAbsolutePath val path2 = TestUtils.tempRelativeDir("data2").getAbsolutePath - if (enableRemoteStorage) { - props.put("log.dirs", path1) - props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, enableRemoteStorage.toString) - } else { - props.put("log.dirs", path1 + "," + path2) - } + props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, enableRemoteStorage.toString) + props.put("log.dirs", path1 + "," + path2) propsModifier.apply(props) val config = KafkaConfig.fromProps(props) val logProps = new Properties() @@ -3319,7 +3390,7 @@ class ReplicaManagerTest { when(mockLog.remoteLogEnabled()).thenReturn(enableRemoteStorage) when(mockLog.remoteStorageSystemEnable).thenReturn(enableRemoteStorage) val aliveBrokers = aliveBrokerIds.map(brokerId => new Node(brokerId, s"host$brokerId", brokerId)) - brokerTopicStats = new BrokerTopicStats(java.util.Optional.of(KafkaConfig.fromProps(props))) + brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).remoteLogManagerConfig.enableRemoteStorageSystem) val metadataCache: MetadataCache = mock(classOf[MetadataCache]) when(metadataCache.topicIdInfo()).thenReturn((topicIds.asJava, topicNames.asJava)) @@ -4025,7 +4096,7 @@ class ReplicaManagerTest { val config = new AbstractConfig(RemoteLogManagerConfig.CONFIG_DEF, props) val remoteLogManagerConfig = new RemoteLogManagerConfig(config) val mockLog = mock(classOf[UnifiedLog]) - val brokerTopicStats = new BrokerTopicStats(java.util.Optional.of(KafkaConfig.fromProps(props))) + val brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).remoteLogManagerConfig.enableRemoteStorageSystem()) val remoteLogManager = new RemoteLogManager( remoteLogManagerConfig, 0, @@ -4034,7 +4105,9 @@ class ReplicaManagerTest { time, _ => Optional.of(mockLog), (TopicPartition, Long) => {}, - brokerTopicStats) + brokerTopicStats, + metrics) + remoteLogManager.startup() val spyRLM = spy(remoteLogManager) val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), aliveBrokerIds = Seq(0, 1, 2), enableRemoteStorage = true, shouldMockLog = true, remoteLogManager = Some(spyRLM)) @@ -4123,7 +4196,7 @@ class ReplicaManagerTest { val config = new AbstractConfig(RemoteLogManagerConfig.CONFIG_DEF, props) val remoteLogManagerConfig = new RemoteLogManagerConfig(config) val dummyLog = mock(classOf[UnifiedLog]) - val brokerTopicStats = new BrokerTopicStats(java.util.Optional.of(KafkaConfig.fromProps(props))) + val brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).remoteLogManagerConfig.enableRemoteStorageSystem()) val remoteLogManager = new RemoteLogManager( remoteLogManagerConfig, 0, @@ -4132,7 +4205,9 @@ class ReplicaManagerTest { time, _ => Optional.of(dummyLog), (TopicPartition, Long) => {}, - brokerTopicStats) + brokerTopicStats, + metrics) + remoteLogManager.startup() val spyRLM = spy(remoteLogManager) val timer = new MockTimer(time) @@ -4983,9 +5058,8 @@ class ReplicaManagerTest { assertEquals(followerPartitions, actualFollowerPartitions) } - // KAFKA-16031: Enabling remote storage after JBOD is supported in tiered storage @ParameterizedTest - @ValueSource(booleans = Array(false)) + @ValueSource(booleans = Array(true, false)) def testApplyDeltaShouldHandleReplicaAssignedToOnlineDirectory(enableRemoteStorage: Boolean): Unit = { val localId = 1 val topicPartition0 = new TopicPartition("foo", 0) @@ -6377,6 +6451,39 @@ class ReplicaManagerTest { assertEquals(Errors.NONE.code, response.errorCode) assertTrue(response.totalBytes > 0) assertTrue(response.usableBytes >= 0) + assertFalse(response.topics().isEmpty) + response.topics().forEach(t => assertFalse(t.partitions().isEmpty)) + } + } finally { + replicaManager.shutdown(checkpointHW = false) + } + } + + @Test + def testDescribeLogDirsWithoutAnyPartitionTopic(): Unit = { + val noneTopic = "none-topic" + val topicPartition = 0 + val topicId = Uuid.randomUuid() + val followerBrokerId = 0 + val leaderBrokerId = 1 + val leaderEpoch = 1 + val leaderEpochIncrement = 2 + val countDownLatch = new CountDownLatch(1) + val offsetFromLeader = 5 + + // Prepare the mocked components for the test + val (replicaManager, mockLogMgr) = prepareReplicaManagerAndLogManager(new MockTimer(time), + topicPartition, leaderEpoch + leaderEpochIncrement, followerBrokerId, leaderBrokerId, countDownLatch, + expectTruncation = false, localLogOffset = Some(10), offsetFromLeader = offsetFromLeader, topicId = Some(topicId)) + + try { + val responses = replicaManager.describeLogDirs(Set(new TopicPartition(noneTopic, topicPartition))) + assertEquals(mockLogMgr.liveLogDirs.size, responses.size) + responses.foreach { response => + assertEquals(Errors.NONE.code, response.errorCode) + assertTrue(response.totalBytes > 0) + assertTrue(response.usableBytes >= 0) + assertTrue(response.topics().isEmpty) } } finally { replicaManager.shutdown(checkpointHW = false) diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala index 8b4bb93a66c7f..03623bab41f54 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala @@ -299,7 +299,7 @@ class ReplicationQuotasTest extends QuorumTestHarness { features.add(new BrokerRegistrationRequestData.Feature() .setName(MetadataVersion.FEATURE_NAME) .setMinSupportedVersion(MetadataVersion.IBP_3_0_IV1.featureLevel()) - .setMaxSupportedVersion(MetadataVersion.IBP_3_8_IV0.featureLevel())) + .setMaxSupportedVersion(MetadataVersion.IBP_4_0_IVO.featureLevel())) controllerServer.controller.registerBroker( ControllerRequestContextUtil.ANONYMOUS_CONTEXT, new BrokerRegistrationRequestData() diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 7314d9ed612b7..97efd9bcf4cc0 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -20,6 +20,7 @@ import kafka.utils.TestUtils import org.apache.kafka.common._ import org.apache.kafka.common.acl._ import org.apache.kafka.common.compress.Compression +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.config.{ConfigResource, TopicConfig} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic @@ -45,7 +46,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.metadata.authorizer.StandardAuthorizer import org.apache.kafka.network.Session import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult} -import org.apache.kafka.server.config.{KafkaSecurityConfigs, QuotaConfigs} +import org.apache.kafka.server.config.{ServerConfigs, QuotaConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -78,23 +79,23 @@ class RequestQuotaTest extends BaseRequestTest { private val tasks = new ListBuffer[Task] override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.ControlledShutdownEnableProp, "false") + properties.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1") properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.put(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, "100") properties.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") - properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[RequestQuotaTest.TestPrincipalBuilder].getName) - properties.put(KafkaConfig.UnstableApiVersionsEnableProp, "true") + properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[RequestQuotaTest.TestPrincipalBuilder].getName) + properties.put(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") if (isKRaftTest()) { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[RequestQuotaTest.KraftTestAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[RequestQuotaTest.KraftTestAuthorizer].getName) } else { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[RequestQuotaTest.ZkTestAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[RequestQuotaTest.ZkTestAuthorizer].getName) } } override def kraftControllerConfigs(): Seq[Properties] = { val properties = new Properties() - properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[RequestQuotaTest.TestPrincipalBuilder].getName) + properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[RequestQuotaTest.TestPrincipalBuilder].getName) Seq(properties) } @@ -703,10 +704,10 @@ class RequestQuotaTest extends BaseRequestTest { new ConsumerGroupDescribeRequest.Builder(new ConsumerGroupDescribeRequestData(), true) case ApiKeys.GET_TELEMETRY_SUBSCRIPTIONS => - new GetTelemetrySubscriptionsRequest.Builder(new GetTelemetrySubscriptionsRequestData(), true) + new GetTelemetrySubscriptionsRequest.Builder(new GetTelemetrySubscriptionsRequestData()) case ApiKeys.PUSH_TELEMETRY => - new PushTelemetryRequest.Builder(new PushTelemetryRequestData(), true) + new PushTelemetryRequest.Builder(new PushTelemetryRequestData()) case ApiKeys.ASSIGN_REPLICAS_TO_DIRS => new AssignReplicasToDirsRequest.Builder(new AssignReplicasToDirsRequestData()) @@ -717,6 +718,18 @@ class RequestQuotaTest extends BaseRequestTest { case ApiKeys.DESCRIBE_TOPIC_PARTITIONS => new DescribeTopicPartitionsRequest.Builder(new DescribeTopicPartitionsRequestData()) + case ApiKeys.SHARE_GROUP_HEARTBEAT => + new ShareGroupHeartbeatRequest.Builder(new ShareGroupHeartbeatRequestData(), true) + + case ApiKeys.SHARE_GROUP_DESCRIBE => + new ShareGroupDescribeRequest.Builder(new ShareGroupDescribeRequestData(), true) + + case ApiKeys.SHARE_FETCH => + new ShareFetchRequest.Builder(new ShareFetchRequestData(), true) + + case ApiKeys.SHARE_ACKNOWLEDGE => + new ShareAcknowledgeRequest.Builder(new ShareAcknowledgeRequestData(), true) + case _ => throw new IllegalArgumentException("Unsupported API key " + apiKey) } diff --git a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala index 493b1c216d8e1..1ea720d697af9 100644 --- a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala @@ -31,7 +31,6 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse, SaslHandshakeRequest, SaslHandshakeResponse} import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.network.SocketServerConfigs -import org.apache.kafka.server.config.KafkaSecurityConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.extension.ExtendWith import org.junit.jupiter.api.{AfterEach, BeforeEach} @@ -46,7 +45,7 @@ object SaslApiVersionsRequestTest { def saslApiVersionsRequestClusterConfig(): java.util.List[ClusterConfig] = { val saslServerProperties = new java.util.HashMap[String, String]() - saslServerProperties.put(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, kafkaClientSaslMechanism) + saslServerProperties.put(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, kafkaClientSaslMechanism) saslServerProperties.put(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, kafkaServerSaslMechanisms.mkString(",")) val saslClientProperties = new java.util.HashMap[String, String]() diff --git a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala index a4fa2b315e5be..6da56191b4df6 100755 --- a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala @@ -19,6 +19,7 @@ package kafka.server import kafka.utils.TestUtils import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, PropertiesUtils} +import org.apache.kafka.server.config.ServerConfigs import org.apache.zookeeper.KeeperException.NodeExistsException import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -87,9 +88,9 @@ class ServerGenerateBrokerIdTest extends QuorumTestHarness { @Test def testDisableGeneratedBrokerId(): Unit = { val props3 = TestUtils.createBrokerConfig(3, zkConnect) - props3.put(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props3.put(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") // Set reserve broker ids to cause collision and ensure disabling broker id generation ignores the setting - props3.put(KafkaConfig.MaxReservedBrokerIdProp, "0") + props3.put(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG, "0") val config3 = KafkaConfig.fromProps(props3) val server3 = createServer(config3, threadNamePrefix = Option(this.getClass.getName)) servers = Seq(server3) @@ -151,14 +152,14 @@ class ServerGenerateBrokerIdTest extends QuorumTestHarness { servers = Seq(serverA) // adjust the broker config and start again - propsB.setProperty(KafkaConfig.BrokerIdProp, "2") + propsB.setProperty(ServerConfigs.BROKER_ID_CONFIG, "2") val serverB2 = new KafkaServer(KafkaConfig.fromProps(propsB), threadNamePrefix = Option(this.getClass.getName)) val startupException = assertThrows(classOf[RuntimeException], () => serverB2.startup()) assertTrue(startupException.getMessage.startsWith("Stored node id 1 doesn't match previous node id 2"), "Unexpected exception message " + startupException.getMessage) serverB2.config.logDirs.foreach(logDir => Utils.delete(new File(logDir))) - propsB.setProperty(KafkaConfig.BrokerIdProp, "3") + propsB.setProperty(ServerConfigs.BROKER_ID_CONFIG, "3") val serverB3 = new KafkaServer(KafkaConfig.fromProps(propsB), threadNamePrefix = Option(this.getClass.getName)) serverB3.startup() diff --git a/core/src/test/scala/unit/kafka/server/ServerTest.scala b/core/src/test/scala/unit/kafka/server/ServerTest.scala index 8c36aa3464a00..62345c446e27d 100644 --- a/core/src/test/scala/unit/kafka/server/ServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerTest.scala @@ -17,11 +17,10 @@ package kafka.server import java.util.Properties - import org.apache.kafka.common.Uuid import org.apache.kafka.common.metrics.MetricsContext import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.config.{KRaftConfigs, ZkConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ZkConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -55,7 +54,7 @@ class ServerTest { val clusterId = Uuid.randomUuid().toString val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, brokerId.toString) + props.put(ServerConfigs.BROKER_ID_CONFIG, brokerId.toString) props.put(ZkConfigs.ZK_CONNECT_CONFIG, "127.0.0.1:0") val config = KafkaConfig.fromProps(props) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherTierStateMachineTest.scala b/core/src/test/scala/unit/kafka/server/TierStateMachineTest.scala similarity index 93% rename from core/src/test/scala/unit/kafka/server/ReplicaFetcherTierStateMachineTest.scala rename to core/src/test/scala/unit/kafka/server/TierStateMachineTest.scala index af7d1ce633f13..139aeb053ffea 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherTierStateMachineTest.scala +++ b/core/src/test/scala/unit/kafka/server/TierStateMachineTest.scala @@ -23,20 +23,21 @@ import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.record._ import org.apache.kafka.common.{TopicPartition, Uuid} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test import kafka.server.FetcherThreadTestUtils.{initialFetchState, mkBatch} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.collection.Map -class ReplicaFetcherTierStateMachineTest { +class TierStateMachineTest { - val truncateOnFetch = true val topicIds = Map("topic1" -> Uuid.randomUuid(), "topic2" -> Uuid.randomUuid()) val version = ApiKeys.FETCH.latestVersion() private val failedPartitions = new FailedPartitions - @Test - def testFollowerFetchMovedToTieredStore(): Unit = { + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testFollowerFetchMovedToTieredStore(truncateOnFetch: Boolean): Unit = { val partition = new TopicPartition("topic", 0) val replicaLog = Seq( @@ -94,8 +95,9 @@ class ReplicaFetcherTierStateMachineTest { * tiered storage as well. Hence, `X < globalLogStartOffset`. * 4. Follower comes online and tries to fetch X from leader. */ - @Test - def testFollowerFetchOffsetOutOfRangeWithTieredStore(): Unit = { + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testFollowerFetchOffsetOutOfRangeWithTieredStore(truncateOnFetch: Boolean): Unit = { val partition = new TopicPartition("topic", 0) val replicaLog = Seq( @@ -105,7 +107,7 @@ class ReplicaFetcherTierStateMachineTest { val replicaState = PartitionState(replicaLog, leaderEpoch = 7, highWatermark = 0L, rlmEnabled = true) - val mockLeaderEndpoint = new MockLeaderEndPoint + val mockLeaderEndpoint = new MockLeaderEndPoint(truncateOnFetch = truncateOnFetch, version = version) val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) val fetcher = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine) @@ -153,8 +155,9 @@ class ReplicaFetcherTierStateMachineTest { assertEquals(11L, replicaState.logEndOffset) } - @Test - def testFencedOffsetResetAfterMovedToRemoteTier(): Unit = { + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testFencedOffsetResetAfterMovedToRemoteTier(truncateOnFetch: Boolean): Unit = { val partition = new TopicPartition("topic", 0) var isErrorHandled = false val mockLeaderEndpoint = new MockLeaderEndPoint(truncateOnFetch = truncateOnFetch, version = version) @@ -189,4 +192,5 @@ class ReplicaFetcherTierStateMachineTest { assertTrue(fetcher.fetchState(partition).isEmpty) assertTrue(failedPartitions.contains(partition)) } -} + +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/TopicIdWithOldInterBrokerProtocolTest.scala b/core/src/test/scala/unit/kafka/server/TopicIdWithOldInterBrokerProtocolTest.scala index f41487c5d4ad3..ea3f79ddbce3a 100644 --- a/core/src/test/scala/unit/kafka/server/TopicIdWithOldInterBrokerProtocolTest.scala +++ b/core/src/test/scala/unit/kafka/server/TopicIdWithOldInterBrokerProtocolTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{DeleteTopicsRequest, DeleteTopicsResponse, FetchRequest, FetchResponse, MetadataRequest, MetadataResponse} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.server.common.MetadataVersion.IBP_2_7_IV0 -import org.apache.kafka.server.config.ReplicationConfigs +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} @@ -41,7 +41,7 @@ class TopicIdWithOldInterBrokerProtocolTest extends BaseRequestTest { properties.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, IBP_2_7_IV0.toString) properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2") - properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") + properties.setProperty(ServerConfigs.BROKER_RACK_CONFIG, s"rack/${properties.getProperty(ServerConfigs.BROKER_ID_CONFIG)}") } @BeforeEach 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 c2926c3b67db9..26f4fb3daee8c 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala @@ -30,7 +30,6 @@ import org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET 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.metadata.FeatureLevelRecord import org.apache.kafka.common.utils.Exit import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataImageTest, MetadataProvenance} @@ -43,7 +42,7 @@ import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertTrue import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any import org.mockito.Mockito -import org.mockito.Mockito.{clearInvocations, doThrow, mock, times, verify, verifyNoInteractions} +import org.mockito.Mockito.{doThrow, mock, verify} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -221,102 +220,4 @@ class BrokerMetadataPublisherTest { verify(groupCoordinator).onNewMetadataImage(image, delta) } - - @Test - def testMetadataVersionUpdateToIBP_3_7_IV2OrAboveTriggersBrokerReRegistration(): Unit = { - val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, "")) - val metadataCache = new KRaftMetadataCache(0) - val logManager = mock(classOf[LogManager]) - val replicaManager = mock(classOf[ReplicaManager]) - val groupCoordinator = mock(classOf[GroupCoordinator]) - val faultHandler = mock(classOf[FaultHandler]) - val brokerLifecycleManager = mock(classOf[BrokerLifecycleManager]) - - val metadataPublisher = new BrokerMetadataPublisher( - config, - metadataCache, - logManager, - replicaManager, - groupCoordinator, - mock(classOf[TransactionCoordinator]), - mock(classOf[DynamicConfigPublisher]), - mock(classOf[DynamicClientQuotaPublisher]), - mock(classOf[ScramPublisher]), - mock(classOf[DelegationTokenPublisher]), - mock(classOf[AclPublisher]), - faultHandler, - faultHandler, - brokerLifecycleManager, - ) - - var image = MetadataImage.EMPTY - var delta = new MetadataDelta.Builder() - .setImage(image) - .build() - - // We first upgrade metadata version to 3_6_IV2 - delta.replay(new FeatureLevelRecord(). - setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(MetadataVersion.IBP_3_6_IV2.featureLevel())) - var newImage = delta.apply(new MetadataProvenance(100, 4, 2000)) - - metadataPublisher.onMetadataUpdate(delta, newImage, - LogDeltaManifest.newBuilder() - .provenance(MetadataProvenance.EMPTY) - .leaderAndEpoch(LeaderAndEpoch.UNKNOWN) - .numBatches(1) - .elapsedNs(100) - .numBytes(42) - .build()) - - // This should NOT trigger broker reregistration - verifyNoInteractions(brokerLifecycleManager) - - // We then upgrade to IBP_3_7_IV2 - image = newImage - delta = new MetadataDelta.Builder() - .setImage(image) - .build() - delta.replay(new FeatureLevelRecord(). - setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(MetadataVersion.IBP_3_7_IV2.featureLevel())) - newImage = delta.apply(new MetadataProvenance(100, 4, 2000)) - - metadataPublisher.onMetadataUpdate(delta, newImage, - LogDeltaManifest.newBuilder() - .provenance(MetadataProvenance.EMPTY) - .leaderAndEpoch(LeaderAndEpoch.UNKNOWN) - .numBatches(1) - .elapsedNs(100) - .numBytes(42) - .build()) - - // This SHOULD trigger a broker registration - verify(brokerLifecycleManager, times(1)).handleKraftJBODMetadataVersionUpdate() - clearInvocations(brokerLifecycleManager) - - // Finally upgrade to IBP_3_8_IV0 - image = newImage - delta = new MetadataDelta.Builder() - .setImage(image) - .build() - delta.replay(new FeatureLevelRecord(). - setName(MetadataVersion.FEATURE_NAME). - setFeatureLevel(MetadataVersion.IBP_3_8_IV0.featureLevel())) - newImage = delta.apply(new MetadataProvenance(200, 4, 3000)) - - metadataPublisher.onMetadataUpdate(delta, newImage, - LogDeltaManifest.newBuilder() - .provenance(MetadataProvenance.EMPTY) - .leaderAndEpoch(LeaderAndEpoch.UNKNOWN) - .numBatches(1) - .elapsedNs(100) - .numBytes(42) - .build()) - - // This should NOT trigger broker reregistration - verify(brokerLifecycleManager, times(0)).handleKraftJBODMetadataVersionUpdate() - - metadataPublisher.close() - } } diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index de7736973b2b4..b8764f5fae3d0 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -28,7 +28,7 @@ import kafka.log.{LogTestUtils, UnifiedLog} import kafka.raft.{KafkaMetadataLog, MetadataLogConfig} import kafka.server.{BrokerTopicStats, KafkaRaftServer} import kafka.tools.DumpLogSegments.{OffsetsMessageParser, TimeIndexDumpErrors} -import kafka.utils.TestUtils +import kafka.utils.{TestUtils, VerifiableProperties} import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.{Assignment, Subscription} import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.{TopicPartition, Uuid} @@ -598,6 +598,26 @@ class DumpLogSegmentsTest { ) } + @Test + def testNewDecoder(): Unit = { + // Decoder translate should pass without exception + DumpLogSegments.newDecoder(classOf[DumpLogSegmentsTest.TestDecoder].getName) + DumpLogSegments.newDecoder(classOf[kafka.serializer.DefaultDecoder].getName) + assertThrows(classOf[Exception], () => DumpLogSegments.newDecoder(classOf[DumpLogSegmentsTest.TestDecoderWithoutVerifiableProperties].getName)) + } + + @Test + def testConvertDeprecatedDecoderClass(): Unit = { + assertEquals(classOf[org.apache.kafka.tools.api.DefaultDecoder].getName, DumpLogSegments.convertDeprecatedDecoderClass( + classOf[kafka.serializer.DefaultDecoder].getName)) + assertEquals(classOf[org.apache.kafka.tools.api.IntegerDecoder].getName, DumpLogSegments.convertDeprecatedDecoderClass( + classOf[kafka.serializer.IntegerDecoder].getName)) + assertEquals(classOf[org.apache.kafka.tools.api.LongDecoder].getName, DumpLogSegments.convertDeprecatedDecoderClass( + classOf[kafka.serializer.LongDecoder].getName)) + assertEquals(classOf[org.apache.kafka.tools.api.StringDecoder].getName, DumpLogSegments.convertDeprecatedDecoderClass( + classOf[kafka.serializer.StringDecoder].getName)) + } + private def readBatchMetadata(lines: util.ListIterator[String]): Option[String] = { while (lines.hasNext) { val line = lines.next() @@ -732,3 +752,13 @@ class DumpLogSegmentsTest { } } } + +object DumpLogSegmentsTest { + class TestDecoder(props: VerifiableProperties) extends kafka.serializer.Decoder[Array[Byte]] { + override def fromBytes(bytes: Array[Byte]): Array[Byte] = bytes + } + + class TestDecoderWithoutVerifiableProperties() extends kafka.serializer.Decoder[Array[Byte]] { + override def fromBytes(bytes: Array[Byte]): Array[Byte] = bytes + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index 896e7169671ff..54a436f231fc4 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -21,25 +21,27 @@ import java.io.{ByteArrayOutputStream, PrintStream} import java.nio.charset.StandardCharsets import java.nio.file.{Files, Paths} import java.util -import java.util.Properties +import java.util.{Collections, Properties} import org.apache.kafka.common.{DirectoryId, KafkaException} import kafka.server.KafkaConfig import kafka.utils.Exit import kafka.utils.TestUtils +import net.sourceforge.argparse4j.inf.Namespace import org.apache.commons.io.output.NullOutputStream import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.common.metadata.UserScramCredentialRecord +import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion, TestFeatureVersion} +import org.apache.kafka.common.metadata.{FeatureLevelRecord, UserScramCredentialRecord} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue} import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.params.provider.{EnumSource, ValueSource} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.jdk.CollectionConverters._ @Timeout(value = 40) class StorageToolTest { @@ -54,6 +56,8 @@ class StorageToolTest { properties } + val allFeatures = Features.FEATURES.toList + @Test def testConfigToLogDirectories(): Unit = { val config = new KafkaConfig(newSelfManagedProperties()) @@ -206,12 +210,15 @@ Found problem: @Test def testFormatSucceedsIfAllDirectoriesAreAvailable(): Unit = { - val availableDir1 = TestUtils.tempDir() - val availableDir2 = TestUtils.tempDir() + val availableDirs = Seq(TestUtils.tempDir(), TestUtils.tempDir(), TestUtils.tempDir()).map(dir => dir.toString) val stream = new ByteArrayOutputStream() - assertEquals(0, runFormatCommand(stream, Seq(availableDir1.toString, availableDir2.toString))) - assertTrue(stream.toString().contains("Formatting %s".format(availableDir1))) - assertTrue(stream.toString().contains("Formatting %s".format(availableDir2))) + assertEquals(0, runFormatCommand(stream, availableDirs)) + val actual = stream.toString().split("\\r?\\n") + val expect = availableDirs.map("Formatting %s".format(_)) + assertEquals(availableDirs.size, actual.size) + expect.foreach(dir => { + assertEquals(1, actual.count(_.startsWith(dir))) + }) } @Test @@ -259,7 +266,7 @@ Found problem: @Test def testDefaultMetadataVersion(): Unit = { val namespace = StorageTool.parseArguments(Array("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ")) - val mv = StorageTool.getMetadataVersion(namespace, defaultVersionString = None) + val mv = StorageTool.getMetadataVersion(namespace, Map.empty, defaultVersionString = None) assertEquals(MetadataVersion.LATEST_PRODUCTION.featureLevel(), mv.featureLevel(), "Expected the default metadata.version to be the latest production version") } @@ -267,18 +274,58 @@ Found problem: @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)) + val mv = StorageTool.getMetadataVersion(namespace, Map.empty, 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 testSettingFeatureAndReleaseVersionFails(): Unit = { + val namespace = StorageTool.parseArguments(Array("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ", + "--release-version", "3.0-IV1", "--feature", "metadata.version=4")) + assertThrows(classOf[IllegalArgumentException], () => StorageTool.getMetadataVersion(namespace, parseFeatures(namespace), defaultVersionString = None)) + } + + @Test + def testParseFeatures(): Unit = { + def parseAddFeatures(strings: String*): Map[String, java.lang.Short] = { + var args = mutable.Seq("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ") + args ++= strings + val namespace = StorageTool.parseArguments(args.toArray) + parseFeatures(namespace) + } + + assertThrows(classOf[RuntimeException], () => parseAddFeatures("--feature", "blah")) + assertThrows(classOf[RuntimeException], () => parseAddFeatures("--feature", "blah=blah")) + + // Test with no features + assertEquals(Map(), parseAddFeatures()) + + // Test with one feature + val testFeatureLevel = 1 + val testArgument = TestFeatureVersion.FEATURE_NAME + "=" + testFeatureLevel.toString + val expectedMap = Map(TestFeatureVersion.FEATURE_NAME -> testFeatureLevel.toShort) + assertEquals(expectedMap, parseAddFeatures("--feature", testArgument)) + + // Test with two features + val metadataFeatureLevel = 5 + val metadataArgument = MetadataVersion.FEATURE_NAME + "=" + metadataFeatureLevel.toString + val expectedMap2 = expectedMap ++ Map (MetadataVersion.FEATURE_NAME -> metadataFeatureLevel.toShort) + assertEquals(expectedMap2, parseAddFeatures("--feature", testArgument, "--feature", metadataArgument)) + } + + private def parseFeatures(namespace: Namespace): Map[String, java.lang.Short] = { + val specifiedFeatures: util.List[String] = namespace.getList("feature") + StorageTool.featureNamesAndLevels(Option(specifiedFeatures).getOrElse(Collections.emptyList).asScala.toList) + } + @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, defaultVersionString = None) + StorageTool.getMetadataVersion(namespace, Map.empty, defaultVersionString = None) } var mv = parseMetadataVersion("--release-version", "3.0") @@ -290,6 +337,127 @@ Found problem: assertThrows(classOf[IllegalArgumentException], () => parseMetadataVersion("--release-version", "0.0")) } + private def generateRecord(featureName: String, level: Short): ApiMessageAndVersion = { + new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(featureName). + setFeatureLevel(level), 0.toShort) + } + + @ParameterizedTest + @EnumSource(classOf[TestFeatureVersion]) + def testFeatureFlag(testFeatureVersion: TestFeatureVersion): Unit = { + val featureLevel = testFeatureVersion.featureLevel + if (featureLevel <= Features.TEST_VERSION.defaultValue(MetadataVersion.LATEST_PRODUCTION)) { + val records = new ArrayBuffer[ApiMessageAndVersion]() + StorageTool.generateFeatureRecords( + records, + MetadataVersion.LATEST_PRODUCTION, + Map(TestFeatureVersion.FEATURE_NAME -> featureLevel), + allFeatures, + false, + false + ) + if (featureLevel > 0) { + assertEquals(List(generateRecord(TestFeatureVersion.FEATURE_NAME, featureLevel)), records) + } + } + } + + @ParameterizedTest + @EnumSource(classOf[MetadataVersion]) + def testVersionDefault(metadataVersion: MetadataVersion): Unit = { + val records = new ArrayBuffer[ApiMessageAndVersion]() + StorageTool.generateFeatureRecords( + records, + metadataVersion, + Map.empty, + allFeatures, + true, + true + ) + + val expectedRecords = new ArrayBuffer[ApiMessageAndVersion]() + + def maybeAddRecordFor(features: Features): Unit = { + val featureLevel = features.defaultValue(metadataVersion) + if (featureLevel > 0) { + expectedRecords += generateRecord(features.featureName, featureLevel) + } + } + + Features.FEATURES.foreach(maybeAddRecordFor) + + assertEquals(expectedRecords, records) + } + @Test + def testVersionDefaultNoArgs(): Unit = { + val records = new ArrayBuffer[ApiMessageAndVersion]() + StorageTool.generateFeatureRecords( + records, + MetadataVersion.LATEST_PRODUCTION, + Map.empty, + allFeatures, + false, + false + ) + + assertEquals(List(generateRecord(TestFeatureVersion.FEATURE_NAME, Features.TEST_VERSION.defaultValue(MetadataVersion.LATEST_PRODUCTION))), records) + } + + + @Test + def testFeatureDependency(): Unit = { + val featureLevel = 1.toShort + assertThrows(classOf[TerseFailure], () => StorageTool.generateFeatureRecords( + new ArrayBuffer[ApiMessageAndVersion](), + MetadataVersion.IBP_2_8_IV1, + Map(TestFeatureVersion.FEATURE_NAME -> featureLevel), + allFeatures, + false, + false + )) + } + + @Test + def testLatestFeaturesWithOldMetadataVersion(): Unit = { + val records = new ArrayBuffer[ApiMessageAndVersion]() + StorageTool.generateFeatureRecords( + records, + MetadataVersion.IBP_3_3_IV0, + Map.empty, + allFeatures, + false, + false + ) + + assertEquals(List(generateRecord(TestFeatureVersion.FEATURE_NAME, Features.TEST_VERSION.defaultValue(MetadataVersion.LATEST_PRODUCTION))), records) + } + + @Test + def testFeatureInvalidFlag(): Unit = { + val featureLevel = 99.toShort + assertThrows(classOf[IllegalArgumentException], () => StorageTool.generateFeatureRecords( + new ArrayBuffer[ApiMessageAndVersion](), + MetadataVersion.LATEST_PRODUCTION, + Map(TestFeatureVersion.FEATURE_NAME -> featureLevel), + allFeatures, + false, + false + )) + } + + @Test + def testUnstableFeatureThrowsError(): Unit = { + assertThrows(classOf[IllegalArgumentException], () => StorageTool.generateFeatureRecords( + new ArrayBuffer[ApiMessageAndVersion](), + MetadataVersion.LATEST_PRODUCTION, + Map(TestFeatureVersion.FEATURE_NAME -> Features.TEST_VERSION.latestTesting), + allFeatures, + false, + false + )) + } + @Test def testAddScram():Unit = { def parseAddScram(strings: String*): Option[ArrayBuffer[UserScramCredentialRecord]] = { @@ -464,7 +632,7 @@ Found problem: val propsStream = Files.newOutputStream(propsFile.toPath) try { properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, TestUtils.tempDir().toString) - properties.setProperty(KafkaConfig.UnstableMetadataVersionsEnableProp, enableUnstable.toString) + properties.setProperty(ServerConfigs.UNSTABLE_FEATURE_VERSIONS_ENABLE_CONFIG, enableUnstable.toString) properties.store(propsStream, "config.props") } finally { propsStream.close() @@ -489,3 +657,4 @@ Found problem: } } } + diff --git a/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala index e96da4cd7d3a5..8acaa9586104b 100755 --- a/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala @@ -89,19 +89,6 @@ class CoreUtilsTest extends Logging { assertEquals(Integer.MAX_VALUE, Utils.abs(Integer.MAX_VALUE)) } - @Test - def testCsvList(): Unit = { - val emptyString:String = "" - val nullString:String = null - val emptyList = CoreUtils.parseCsvList(emptyString) - val emptyListFromNullString = CoreUtils.parseCsvList(nullString) - val emptyStringList = Seq.empty[String] - assertTrue(emptyList!=null) - assertTrue(emptyListFromNullString!=null) - assertTrue(emptyStringList.equals(emptyListFromNullString)) - assertTrue(emptyStringList.equals(emptyList)) - } - @Test def testInLock(): Unit = { val lock = new ReentrantLock() diff --git a/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala b/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala index 491ac847d4d2c..4a24a8884c9ea 100644 --- a/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala @@ -18,11 +18,10 @@ package kafka.utils import java.io.{BufferedWriter, File, FileWriter} import java.util.Properties - import scala.collection.Seq import org.apache.kafka.clients.admin.ScramMechanism +import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.utils.Java -import org.apache.kafka.server.config.KafkaSecurityConfigs object JaasTestUtils { @@ -161,8 +160,8 @@ object JaasTestUtils { val result = saslProperties.getOrElse(new Properties) // IBM Kerberos module doesn't support the serviceName JAAS property, hence it needs to be // passed as a Kafka property - if (isIbmSecurity && !result.contains(KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG)) - result.put(KafkaSecurityConfigs.SASL_KERBEROS_SERVICE_NAME_CONFIG, serviceName) + if (isIbmSecurity && !result.contains(SaslConfigs.SASL_KERBEROS_SERVICE_NAME)) + result.put(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, serviceName) result } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 8ba2ea00cdd8e..31f671e3a0f4e 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -304,18 +304,18 @@ object TestUtils extends Logging { }.mkString(",") val props = new Properties - props.put(KafkaConfig.UnstableMetadataVersionsEnableProp, "true") + props.put(ServerConfigs.UNSTABLE_FEATURE_VERSIONS_ENABLE_CONFIG, "true") if (zkConnect == null) { props.setProperty(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG, TimeUnit.MINUTES.toMillis(10).toString) props.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString) - props.put(KafkaConfig.BrokerIdProp, nodeId.toString) + props.put(ServerConfigs.BROKER_ID_CONFIG, nodeId.toString) props.put(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) props.put(SocketServerConfigs.LISTENERS_CONFIG, listeners) props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") props.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, protocolAndPorts. map(p => "%s:%s".format(p._1, p._1)).mkString(",") + ",CONTROLLER:PLAINTEXT") } else { - if (nodeId >= 0) props.put(KafkaConfig.BrokerIdProp, nodeId.toString) + if (nodeId >= 0) props.put(ServerConfigs.BROKER_ID_CONFIG, nodeId.toString) props.put(SocketServerConfigs.LISTENERS_CONFIG, listeners) } if (logDirCount > 1) { @@ -341,20 +341,20 @@ object TestUtils extends Logging { } props.put(ReplicationConfigs.REPLICA_SOCKET_TIMEOUT_MS_CONFIG, "1500") props.put(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG, "1500") - props.put(KafkaConfig.ControlledShutdownEnableProp, enableControlledShutdown.toString) - props.put(KafkaConfig.DeleteTopicEnableProp, enableDeleteTopic.toString) + props.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, enableControlledShutdown.toString) + props.put(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, enableDeleteTopic.toString) props.put(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, "1000") - props.put(KafkaConfig.ControlledShutdownRetryBackoffMsProp, "100") + props.put(ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG, "100") props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152") props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1") if (!props.containsKey(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG)) props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "5") if (!props.containsKey(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG)) props.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") - rack.foreach(props.put(KafkaConfig.RackProp, _)) + rack.foreach(props.put(ServerConfigs.BROKER_RACK_CONFIG, _)) // Reduce number of threads per broker - props.put(KafkaConfig.NumNetworkThreadsProp, "2") - props.put(KafkaConfig.BackgroundThreadsProp, "2") + props.put(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, "2") + props.put(ServerConfigs.BACKGROUND_THREADS_CONFIG, "2") if (protocolAndPorts.exists { case (protocol, _) => usesSslTransportLayer(protocol) }) props ++= sslConfigs(Mode.SERVER, false, trustStoreFile, s"server$nodeId") @@ -367,13 +367,13 @@ object TestUtils extends Logging { } if (enableToken) - props.put(KafkaConfig.DelegationTokenSecretKeyProp, "secretkey") + props.put(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "secretkey") props.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numPartitions.toString) props.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, defaultReplicationFactor.toString) if (enableFetchFromFollower) { - props.put(KafkaConfig.RackProp, nodeId.toString) + props.put(ServerConfigs.BROKER_RACK_CONFIG, nodeId.toString) props.put(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG, "org.apache.kafka.common.replica.RackAwareReplicaSelector") } props @@ -1417,6 +1417,33 @@ object TestUtils extends Logging { 45000) } + + def waitAndVerifyAcl(expected: AccessControlEntry, + authorizer: JAuthorizer, + resource: ResourcePattern, + accessControlEntryFilter: AccessControlEntryFilter = AccessControlEntryFilter.ANY): Unit = { + val newLine = scala.util.Properties.lineSeparator + + val filter = new AclBindingFilter(resource.toFilter, accessControlEntryFilter) + waitUntilTrue(() => authorizer.acls(filter).asScala.map(_.entry).toSet.contains(expected), + s"expected to contain acl: $expected" + + s"but got:${authorizer.acls(filter).asScala.map(_.entry).mkString(newLine + "\t", newLine + "\t", newLine)}", + 45000) + } + + def waitAndVerifyRemovedAcl(expectedToRemoved: AccessControlEntry, + authorizer: JAuthorizer, + resource: ResourcePattern, + accessControlEntryFilter: AccessControlEntryFilter = AccessControlEntryFilter.ANY): Unit = { + val newLine = scala.util.Properties.lineSeparator + + val filter = new AclBindingFilter(resource.toFilter, accessControlEntryFilter) + waitUntilTrue(() => !authorizer.acls(filter).asScala.map(_.entry).toSet.contains(expectedToRemoved), + s"expected acl to be removed : $expectedToRemoved" + + s"but got:${authorizer.acls(filter).asScala.map(_.entry).mkString(newLine + "\t", newLine + "\t", newLine)}", + 45000) + } + /** * Verifies that this ACL is the secure one. */ diff --git a/core/src/test/scala/unit/kafka/zk/migration/ZkConfigMigrationClientTest.scala b/core/src/test/scala/unit/kafka/zk/migration/ZkConfigMigrationClientTest.scala index 1173cc7fe3057..b723c5823bf12 100644 --- a/core/src/test/scala/unit/kafka/zk/migration/ZkConfigMigrationClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/migration/ZkConfigMigrationClientTest.scala @@ -21,7 +21,7 @@ import kafka.server.ZkAdminManager import kafka.zk.{AdminZkClient, ZkMigrationClient} import org.apache.kafka.clients.admin.ScramMechanism import org.apache.kafka.common.config.types.Password -import org.apache.kafka.common.config.{ConfigResource, TopicConfig} +import org.apache.kafka.common.config.{ConfigResource, SslConfigs, TopicConfig} import org.apache.kafka.common.metadata.ClientQuotaRecord import org.apache.kafka.common.metadata.ClientQuotaRecord.EntityData import org.apache.kafka.common.metadata.ConfigRecord @@ -37,7 +37,7 @@ import org.apache.kafka.metadata.RecordTestUtils import org.apache.kafka.metadata.migration.KRaftMigrationZkWriter import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState import org.apache.kafka.server.common.ApiMessageAndVersion -import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ReplicationConfigs, QuotaConfigs} +import org.apache.kafka.server.config.{ConfigType, QuotaConfigs, ReplicationConfigs} import org.apache.kafka.server.util.MockRandom import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue, fail} import org.junit.jupiter.api.Test @@ -63,7 +63,7 @@ class ZkConfigMigrationClientTest extends ZkMigrationTestHarness { // Create some configs and persist in Zk. val props = new Properties() props.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "1") // normal config - props.put(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, encoder.encode(new Password(SECRET))) // sensitive config + props.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, encoder.encode(new Password(SECRET))) // sensitive config zkClient.setOrCreateEntityConfigs(ConfigType.BROKER, "1", props) val defaultProps = new Properties() @@ -83,7 +83,7 @@ class ZkConfigMigrationClientTest extends ZkMigrationTestHarness { assertTrue(props.containsKey(name)) // If the config is sensitive, compare it to the decoded value. - if (name == KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) { + if (name == SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) { assertEquals(SECRET, value) } else { assertEquals(props.getProperty(name), value) @@ -100,13 +100,13 @@ class ZkConfigMigrationClientTest extends ZkMigrationTestHarness { // persisted in Zookeeper is encrypted. val newProps = new util.HashMap[String, String]() newProps.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2") // normal config - newProps.put(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, NEW_SECRET) // sensitive config + newProps.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, NEW_SECRET) // sensitive config migrationState = migrationClient.configClient().writeConfigs( new ConfigResource(ConfigResource.Type.BROKER, "1"), newProps, migrationState) val actualPropsInZk = zkClient.getEntityConfigs(ConfigType.BROKER, "1") assertEquals(2, actualPropsInZk.size()) actualPropsInZk.forEach { case (key, value) => - if (key == KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) { + if (key == SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) { assertEquals(NEW_SECRET, encoder.decode(value.toString).value) } else { assertEquals(newProps.get(key), value) diff --git a/core/src/test/scala/unit/kafka/zk/migration/ZkMigrationClientTest.scala b/core/src/test/scala/unit/kafka/zk/migration/ZkMigrationClientTest.scala index e2e5b8b58b944..bc4b5ec694429 100644 --- a/core/src/test/scala/unit/kafka/zk/migration/ZkMigrationClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/migration/ZkMigrationClientTest.scala @@ -19,7 +19,7 @@ package kafka.zk.migration import kafka.api.LeaderAndIsr import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment} import kafka.coordinator.transaction.{ProducerIdManager, ZkProducerIdManager} -import org.apache.kafka.common.config.{ConfigResource, TopicConfig} +import org.apache.kafka.common.config.{ConfigResource, SslConfigs, TopicConfig} import org.apache.kafka.common.errors.ControllerMovedException import org.apache.kafka.common.metadata.{ConfigRecord, MetadataRecordType, PartitionRecord, ProducerIdsRecord, TopicRecord} import org.apache.kafka.common.{DirectoryId, TopicPartition, Uuid} @@ -28,7 +28,7 @@ import org.apache.kafka.metadata.migration.{KRaftMigrationZkWriter, ZkMigrationL import org.apache.kafka.metadata.{LeaderRecoveryState, PartitionRegistration} import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.common.ApiMessageAndVersion -import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs} +import org.apache.kafka.server.config.ConfigType import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue, fail} import org.junit.jupiter.api.Test @@ -331,7 +331,7 @@ class ZkMigrationClientTest extends ZkMigrationTestHarness { val topicId = Uuid.randomUuid() val props = new Properties() props.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "1") // normal config - props.put(KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, SECRET) // sensitive config + props.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, SECRET) // sensitive config // // Leave Zk in an incomplete state. // zkClient.createTopicAssignment(topicName, Some(topicId), Map(tp -> Seq(1))) @@ -402,7 +402,7 @@ class ZkMigrationClientTest extends ZkMigrationTestHarness { assertEquals(2, brokerProps.size()) brokerProps.asScala.foreach { case (key, value) => - if (key == KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) { + if (key == SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) { assertEquals(SECRET, encoder.decode(value).value) } else { assertEquals(props.getProperty(key), value) @@ -410,7 +410,7 @@ class ZkMigrationClientTest extends ZkMigrationTestHarness { } topicProps.asScala.foreach { case (key, value) => - if (key == KafkaSecurityConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) { + if (key == SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) { assertEquals(SECRET, encoder.decode(value).value) } else { assertEquals(props.getProperty(key), value) diff --git a/docker/docker_official_image_build_test.py b/docker/docker_official_image_build_test.py new file mode 100644 index 0000000000000..6ffe25ee0b89c --- /dev/null +++ b/docker/docker_official_image_build_test.py @@ -0,0 +1,87 @@ +#!/usr/bin/env python + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +""" +Python script to build and test a docker image +This script is used to generate a test report + +Usage: + docker_official_image_build_test.py --help + Get detailed description of each option + + Example command:- + docker_official_image_build_test.py --image-tag --image-type --kafka-version + + This command will build an image with as image name, as image_tag (it will be latest by default), + as image type (jvm by default), for the kafka version for which the image is being built, and + run tests on the image. + -b can be passed as additional argument if you just want to build the image. + -t can be passed if you just want to run tests on the image. +""" + +import argparse +from distutils.dir_util import copy_tree +import shutil +from common import execute +from docker_build_test import run_docker_tests +import tempfile +import os + + +def build_docker_official_image(image, tag, kafka_version, image_type): + image = f'{image}:{tag}' + current_dir = os.path.dirname(os.path.realpath(__file__)) + temp_dir_path = tempfile.mkdtemp() + copy_tree(f"{current_dir}/docker_official_images/{kafka_version}/{image_type}", + f"{temp_dir_path}/{image_type}") + copy_tree(f"{current_dir}/docker_official_images/{kafka_version}/jvm/resources", + f"{temp_dir_path}/{image_type}/resources") + command = f"docker build -f $DOCKER_FILE -t {image} $DOCKER_DIR" + command = command.replace("$DOCKER_FILE", f"{temp_dir_path}/{image_type}/Dockerfile") + command = command.replace("$DOCKER_DIR", f"{temp_dir_path}/{image_type}") + try: + execute(command.split()) + except: + raise SystemError("Docker Image Build failed") + finally: + shutil.rmtree(temp_dir_path) + + +if __name__ == '__main__': + parser = argparse.ArgumentParser() + parser.add_argument( + "image", help="Image name that you want to keep for the Docker image") + parser.add_argument("--image-tag", "-tag", default="latest", + dest="tag", help="Image tag that you want to add to the image") + parser.add_argument("--image-type", "-type", choices=[ + "jvm"], default="jvm", dest="image_type", help="Image type you want to build") + parser.add_argument("--kafka-version", "-v", dest="kafka_version", + help="Kafka version for which the source for docker official image is to be built") + parser.add_argument("--build", "-b", action="store_true", dest="build_only", + default=False, help="Only build the image, don't run tests") + parser.add_argument("--test", "-t", action="store_true", dest="test_only", + default=False, help="Only run the tests, don't build the image") + args = parser.parse_args() + kafka_url = f"https://downloads.apache.org/kafka/{args.kafka_version}/kafka_2.13-{args.kafka_version}.tgz" + if args.build_only or not (args.build_only or args.test_only): + if args.kafka_version: + build_docker_official_image(args.image, args.tag, args.kafka_version, args.image_type) + else: + raise ValueError( + "--kafka-version is required argument for jvm docker official image image") + if args.test_only or not (args.build_only or args.test_only): + run_docker_tests(args.image, args.tag, kafka_url, args.image_type) diff --git a/docker/docker_official_images/.gitkeep b/docker/docker_official_images/.gitkeep new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/docker/extract_docker_official_image_artifact.py b/docker/extract_docker_official_image_artifact.py new file mode 100644 index 0000000000000..2d362eb50db94 --- /dev/null +++ b/docker/extract_docker_official_image_artifact.py @@ -0,0 +1,77 @@ +#!/usr/bin/env python + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +""" +Python script to extract docker official images artifact and give it executable permissions +This script is used to extract docker official images artifact and give it executable permissions + +Usage: + extract_docker_official_image_artifact.py --help + Get detailed description of each option + + Example command:- + extract_docker_official_image_artifact.py --path_to_downloaded_artifact + + This command will build an extract the downloaded artifact, and copy the contents to the + docker_official_images directory. If the extracted artifact contents already exist in the + docker_official_images directory , they will be overwritten, else they will be created. + +""" +import os +import argparse +import zipfile +import shutil +from pathlib import Path + +def set_executable_permissions(directory): + for root, _, files in os.walk(directory): + for file in files: + path = os.path.join(root, file) + os.chmod(path, os.stat(path).st_mode | 0o111) + + +def extract_artifact(artifact_path): + docker_official_images_dir = Path(os.path.dirname(os.path.realpath(__file__)), "docker_official_images") + temp_dir = Path('temp_extracted') + try: + if temp_dir.exists(): + shutil.rmtree(temp_dir) + temp_dir.mkdir() + with zipfile.ZipFile(artifact_path, 'r') as zip_ref: + zip_ref.extractall(temp_dir) + artifact_version_dirs = list(temp_dir.iterdir()) + if len(artifact_version_dirs) != 1: + raise Exception("Unexpected contents in the artifact. Exactly one version directory is expected.") + artifact_version_dir = artifact_version_dirs[0] + target_version_dir = Path(os.path.join(docker_official_images_dir, artifact_version_dir.name)) + target_version_dir.mkdir(parents=True, exist_ok=True) + for image_type_dir in artifact_version_dir.iterdir(): + target_image_type_dir = Path(os.path.join(target_version_dir, image_type_dir.name)) + if target_image_type_dir.exists(): + shutil.rmtree(target_image_type_dir) + shutil.copytree(image_type_dir, target_image_type_dir) + set_executable_permissions(target_image_type_dir) + finally: + if temp_dir.exists(): + shutil.rmtree(temp_dir) + +if __name__ == '__main__': + parser = argparse.ArgumentParser() + parser.add_argument("--path_to_downloaded_artifact", "-artifact_path", required=True, + dest="artifact_path", help="Path to zipped artifacy downloaded from github actions workflow.") + args = parser.parse_args() + extract_artifact(args.artifact_path) diff --git a/docker/generate_kafka_pr_template.py b/docker/generate_kafka_pr_template.py new file mode 100644 index 0000000000000..b3f9577cf89b4 --- /dev/null +++ b/docker/generate_kafka_pr_template.py @@ -0,0 +1,92 @@ +#!/usr/bin/env python + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +""" +Python script to prepare the PR template for the docker official image +This script is used to prepare the PR template for the docker official image + +Usage: + Example command:- + generate_kafka_pr_template.py --help + Get detailed description of each option + + generate_kafka_pr_template.py --image-type + + This command will build a PR template for as image type (jvm by default) based docker official image, + on the directories present under docker/docker_official_images. + This PR template will be used to raise a PR in the Docker Official Images Repo. +""" + +import os +import subprocess +import sys +import argparse +from pathlib import Path + + +# Returns the hash of the most recent commit that modified any of the specified files. +def file_commit(*files): + return subprocess.check_output(["git", "log", "-1", "--format=format:%H", "HEAD", "--"] + list(files)).strip().decode('utf-8') + + +# Returns the latest commit hash for all files in a given directory. +def dir_commit(directory): + docker_required_scripts = [str(path) for path in Path(directory).rglob('*') if path.is_file()] + files_to_check = [os.path.join(directory, "Dockerfile")] + docker_required_scripts + return file_commit(*files_to_check) + + +# Split the version string into parts and convert them to integers for version comparision +def get_version_parts(version): + return tuple(int(part) for part in version.name.split('.')) + + +def main(): + parser = argparse.ArgumentParser() + parser.add_argument("--image-type", "-type", choices=[ + "jvm"], default="jvm", dest="image_type", help="Image type you want to build") + args = parser.parse_args() + self = os.path.basename(__file__) + current_dir = os.path.dirname(os.path.abspath(__file__)) + docker_official_images_dir = Path(os.path.join(current_dir, "docker_official_images")) + highest_version = "" + + header = f""" +# This file is generated via https://github.com/apache/kafka/blob/{file_commit(os.path.join(current_dir, self))}/docker/generate_kafka_pr_template.py +Maintainers: The Apache Kafka Project (@ApacheKafka) +GitRepo: https://github.com/apache/kafka.git +""" + print(header) + versions = sorted((d for d in docker_official_images_dir.iterdir() if d.is_dir()), key=get_version_parts, reverse=True) + highest_version = max(versions).name if versions else "" + + for dir in versions: + version = dir.name + tags = version + (", latest" if version == highest_version else "") + commit = dir_commit(dir.joinpath(args.image_type)) + + info = f""" +Tags: {tags} +Architectures: amd64,arm64v8 +GitCommit: {commit} +Directory: ./docker/docker_official_images/{version}/{args.image_type} +""" + print(info.strip(), '\n') + + +if __name__ == "__main__": + main() diff --git a/docker/native/Dockerfile b/docker/native/Dockerfile index 54db837170bc1..36104b76edca2 100644 --- a/docker/native/Dockerfile +++ b/docker/native/Dockerfile @@ -20,40 +20,26 @@ ARG kafka_url WORKDIR /app ENV KAFKA_URL=$kafka_url -COPY native-image-configs native-image-configs +ENV NATIVE_IMAGE_PATH="native-image" +ENV KAFKA_DIR="/app/kafka" +ENV NATIVE_CONFIGS_DIR="/app/native-image-configs" +ENV KAFKA_LIBS_DIR="$KAFKA_DIR/libs" +ENV TARGET_PATH="$KAFKA_DIR/kafka.Kafka" -RUN mkdir kafka; \ +COPY native-image-configs $NATIVE_CONFIGS_DIR +COPY native_command.sh native_command.sh + +RUN mkdir $KAFKA_DIR; \ microdnf install wget; \ wget -nv -O kafka.tgz "$KAFKA_URL"; \ - wget -nv -O kafka.tgz.asc "$kafka_url.asc"; \ - tar xfz kafka.tgz -C kafka --strip-components 1; \ + wget -nv -O kafka.tgz.asc "$KAFKA_URL.asc"; \ + tar xfz kafka.tgz -C $KAFKA_DIR --strip-components 1; \ wget -nv -O KEYS https://downloads.apache.org/kafka/KEYS; \ gpg --import KEYS; \ gpg --batch --verify kafka.tgz.asc kafka.tgz; \ rm kafka.tgz ; \ - cd kafka ; \ # Build the native-binary of the apache kafka using graalVM native-image. - native-image --no-fallback \ - --enable-http \ - --enable-https \ - --allow-incomplete-classpath \ - --report-unsupported-elements-at-runtime \ - --install-exit-handlers \ - --enable-monitoring=jmxserver,jmxclient,heapdump,jvmstat \ - -H:+ReportExceptionStackTraces \ - -H:+EnableAllSecurityServices \ - -H:EnableURLProtocols=http,https \ - -H:AdditionalSecurityProviders=sun.security.jgss.SunProvider \ - -H:ReflectionConfigurationFiles=/app/native-image-configs/reflect-config.json \ - -H:JNIConfigurationFiles=/app/native-image-configs/jni-config.json \ - -H:ResourceConfigurationFiles=/app/native-image-configs/resource-config.json \ - -H:SerializationConfigurationFiles=/app/native-image-configs/serialization-config.json \ - -H:PredefinedClassesConfigurationFiles=/app/native-image-configs/predefined-classes-config.json \ - -H:DynamicProxyConfigurationFiles=/app/native-image-configs/proxy-config.json \ - --verbose \ - -march=compatibility \ - -cp "libs/*" kafka.docker.KafkaDockerWrapper \ - -o kafka.Kafka + /app/native_command.sh $NATIVE_IMAGE_PATH $NATIVE_CONFIGS_DIR $KAFKA_LIBS_DIR $TARGET_PATH FROM alpine:latest diff --git a/docker/native/native-image-configs/resource-config.json b/docker/native/native-image-configs/resource-config.json index d3fb2031ee6ec..121f24b63722e 100644 --- a/docker/native/native-image-configs/resource-config.json +++ b/docker/native/native-image-configs/resource-config.json @@ -25,13 +25,9 @@ }, { "pattern":"\\Qkafka/kafka-version.properties\\E" }, { - "pattern":"\\Qlinux/amd64/libzstd-jni-1.5.5-6.so\\E" + "pattern":"\\Qlinux/amd64/libzstd-jni-1.5.6-3.so\\E" }, { - "pattern":"\\Qlinux/aarch64/libzstd-jni-1.5.5-6.so\\E" - }, { - "pattern":"\\Qlinux/amd64/libzstd-jni-1.5.5-11.so\\E" - }, { - "pattern":"\\Qlinux/aarch64/libzstd-jni-1.5.5-11.so\\E" + "pattern":"\\Qlinux/aarch64/libzstd-jni-1.5.6-3.so\\E" }, { "pattern":"\\Qnet/jpountz/util/linux/amd64/liblz4-java.so\\E" }, { diff --git a/docker/native/native_command.sh b/docker/native/native_command.sh new file mode 100755 index 0000000000000..6d2ef5efa5cbc --- /dev/null +++ b/docker/native/native_command.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# $1 - The path of the GraalVM native-image. This binary is used to compile Java applications ahead-of-time into a standalone native binary. +# $2 - The path of the directory that contains the native-image configuration files. +# $3 - The path of the directory that contains the Apache Kafka libs. +# $4 - The path of the resulting Kafka native binary after the build process. + +$1 --no-fallback \ + --enable-http \ + --enable-https \ + --allow-incomplete-classpath \ + --report-unsupported-elements-at-runtime \ + --install-exit-handlers \ + --enable-monitoring=jmxserver,jmxclient,heapdump,jvmstat \ + -H:+ReportExceptionStackTraces \ + -H:+EnableAllSecurityServices \ + -H:EnableURLProtocols=http,https \ + -H:AdditionalSecurityProviders=sun.security.jgss.SunProvider \ + -H:ReflectionConfigurationFiles="$2"/reflect-config.json \ + -H:JNIConfigurationFiles="$2"/jni-config.json \ + -H:ResourceConfigurationFiles="$2"/resource-config.json \ + -H:SerializationConfigurationFiles="$2"/serialization-config.json \ + -H:PredefinedClassesConfigurationFiles="$2"/predefined-classes-config.json \ + -H:DynamicProxyConfigurationFiles="$2"/proxy-config.json \ + --verbose \ + -march=compatibility \ + -cp "$3/*" kafka.docker.KafkaDockerWrapper \ + -o "$4" diff --git a/docker/prepare_docker_official_image_source.py b/docker/prepare_docker_official_image_source.py new file mode 100644 index 0000000000000..a39915c9e51f2 --- /dev/null +++ b/docker/prepare_docker_official_image_source.py @@ -0,0 +1,68 @@ +#!/usr/bin/env python + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +""" +Python script to prepare the hardcoded source folder for the docker official image +This script is used to prepare the source folder for the docker official image + +Usage: + prepare_docker_official_image_source.py --help + Get detailed description of each option + + Example command:- + prepare_docker_official_image_source.py --image-type --kafka-version + + This command will build a directory with the name as housing the hardcoded static Dockerfile and scripts for + the docker official image, as image type (jvm by default), for the kafka version for which the + image is being built. +""" + +from datetime import date +import argparse +from distutils.dir_util import copy_tree +import os +import shutil + + +def remove_args_and_hardcode_values(file_path, kafka_url): + with open(file_path, 'r') as file: + filedata = file.read() + filedata = filedata.replace("ARG kafka_url", f"ENV kafka_url {kafka_url}") + filedata = filedata.replace( + "ARG build_date", f"ENV build_date {str(date.today())}") + with open(file_path, 'w') as file: + file.write(filedata) + + +if __name__ == '__main__': + parser = argparse.ArgumentParser() + parser.add_argument("--image-type", "-type", choices=[ + "jvm"], default="jvm", dest="image_type", help="Image type you want to build") + parser.add_argument("--kafka-version", "-v", dest="kafka_version", + help="Kafka version for which the source for docker official image is to be built") + args = parser.parse_args() + kafka_url = f"https://downloads.apache.org/kafka/{args.kafka_version}/kafka_2.13-{args.kafka_version}.tgz" + current_dir = os.path.dirname(os.path.realpath(__file__)) + new_dir = os.path.join( + current_dir, f'docker_official_images', args.kafka_version) + if os.path.exists(new_dir): + shutil.rmtree(new_dir) + os.makedirs(new_dir) + copy_tree(os.path.join(current_dir, args.image_type), os.path.join(new_dir, args.kafka_version, args.image_type)) + copy_tree(os.path.join(current_dir, 'resources'), os.path.join(new_dir, args.kafka_version, args.image_type, 'resources')) + remove_args_and_hardcode_values( + os.path.join(new_dir, args.kafka_version, args.image_type, 'Dockerfile'), kafka_url) diff --git a/docs/js/templateData.js b/docs/js/templateData.js index fa87b7941d0d7..13d352fc70d1d 100644 --- a/docs/js/templateData.js +++ b/docs/js/templateData.js @@ -17,8 +17,8 @@ limitations under the License. // Define variables for doc templates var context={ - "version": "38", - "dotVersion": "3.8", - "fullDotVersion": "3.8.0", + "version": "39", + "dotVersion": "3.9", + "fullDotVersion": "3.9.0", "scalaVersion": "2.13" }; diff --git a/docs/ops.html b/docs/ops.html index c3abea5e0140d..1210b26739f50 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -3244,7 +3244,7 @@
    +

    Upgrading to 3.8.0 from any version 0.8.x through 3.6.x

    + +
    Notable changes in 3.8.0
    +
      +
    • MirrorMaker 2 can now emit checkpoints for offsets mirrored before the start of the Checkpoint task for improved offset translation. + This requires MirrorMaker 2 to have READ authorization to the Checkpoint topic. + If READ is not authorized, checkpointing is limited to offsets mirrorred after the start of the task. + See KAFKA-15905 for more details. +
    • +
    +

    Upgrading to 3.7.1 from any version 0.8.x through 3.6.x

    @@ -83,6 +94,15 @@
    Upgrading KRaft-based cl Given your current and target versions, a downgrade is only possible if there are no metadata changes in the versions between. +
    Notable changes in 3.7.1
    +
      +
    • MirrorMaker 2 can now emit checkpoints for offsets mirrored before the start of the Checkpoint task for improved offset translation. + This requires MirrorMaker 2 to have READ authorization to the Checkpoint topic. + If READ is not authorized, checkpointing is limited to offsets mirrorred after the start of the task. + See KAFKA-15905 for more details. +
    • +
    +
    Notable changes in 3.7.0
    • Java 11 support for the broker and tools has been deprecated and will be removed in Apache Kafka 4.0. This complements diff --git a/gradle.properties b/gradle.properties index e1809d04723a4..88d63be550942 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,7 +23,7 @@ group=org.apache.kafka # - streams/quickstart/pom.xml # - streams/quickstart/java/src/main/resources/archetype-resources/pom.xml # - streams/quickstart/java/pom.xml -version=3.8.0-SNAPSHOT +version=3.9.0-SNAPSHOT scalaVersion=2.13.14 # Adding swaggerVersion in gradle.properties to have a single version in place for swagger # New version of Swagger 2.2.14 requires minimum JDK 11. diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 29732cc1b80d1..f7b6cef051d9c 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -103,10 +103,10 @@ versions += [ gradle: "8.7", grgit: "4.1.1", httpclient: "4.5.14", - jackson: "2.16.1", + jackson: "2.16.2", jacoco: "0.8.10", javassist: "3.29.2-GA", - jetty: "9.4.53.v20231009", + jetty: "9.4.54.v20240208", jersey: "2.39.1", jline: "3.25.1", jmh: "1.37", @@ -146,7 +146,7 @@ versions += [ lz4: "1.8.0", mavenArtifact: "3.9.6", metrics: "2.2.0", - netty: "4.1.100.Final", + netty: "4.1.110.Final", opentelemetryProto: "1.0.0-alpha", pcollections: "4.0.1", powermock: "2.0.9", @@ -165,7 +165,7 @@ versions += [ spotbugs: "4.8.0", zinc: "1.9.2", zookeeper: "3.8.4", - zstd: "1.5.5-11" + zstd: "1.5.6-3" ] libs += [ diff --git a/gradle/spotbugs-exclude.xml b/gradle/spotbugs-exclude.xml index 7c4b96f077d4c..cfa942ed053fd 100644 --- a/gradle/spotbugs-exclude.xml +++ b/gradle/spotbugs-exclude.xml @@ -298,6 +298,13 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read + + + + + + + diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/CoordinatorRecordHelpers.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/CoordinatorRecordHelpers.java index 4cc68229f042b..55766ac1a643a 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/CoordinatorRecordHelpers.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/CoordinatorRecordHelpers.java @@ -65,6 +65,8 @@ public static CoordinatorRecord newMemberSubscriptionRecord( String groupId, ConsumerGroupMember member ) { + List topicNames = new ArrayList<>(member.subscribedTopicNames()); + Collections.sort(topicNames); return new CoordinatorRecord( new ApiMessageAndVersion( new ConsumerGroupMemberMetadataKey() @@ -78,7 +80,7 @@ public static CoordinatorRecord newMemberSubscriptionRecord( .setInstanceId(member.instanceId()) .setClientId(member.clientId()) .setClientHost(member.clientHost()) - .setSubscribedTopicNames(member.subscribedTopicNames()) + .setSubscribedTopicNames(topicNames) .setSubscribedTopicRegex(member.subscribedTopicRegex()) .setServerAssignor(member.serverAssignorName().orElse(null)) .setRebalanceTimeoutMs(member.rebalanceTimeoutMs()) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java index be4e9c0bb3a37..7939dfa630e0b 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java @@ -17,7 +17,7 @@ package org.apache.kafka.coordinator.group; import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; +import org.apache.kafka.coordinator.group.assignor.ConsumerGroupPartitionAssignor; import org.apache.kafka.coordinator.group.assignor.RangeAssignor; import org.apache.kafka.coordinator.group.assignor.UniformAssignor; @@ -182,7 +182,7 @@ public class GroupCoordinatorConfig { /** * The consumer group assignors. */ - public final List consumerGroupAssignors; + public final List consumerGroupAssignors; /** * The offsets topic segment bytes should be kept relatively small to facilitate faster @@ -262,7 +262,7 @@ public GroupCoordinatorConfig( int consumerGroupSessionTimeoutMs, int consumerGroupHeartbeatIntervalMs, int consumerGroupMaxSize, - List consumerGroupAssignors, + List consumerGroupAssignors, int offsetsTopicSegmentBytes, int offsetMetadataMaxSize, int classicGroupMaxSize, diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java index 6fe7928dc750c..5e4e899faa681 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java @@ -423,12 +423,11 @@ public CompletableFuture heartbeat( ); } - // Using a read operation is okay here as we ignore the last committed offset in the snapshot registry. - // This means we will read whatever is in the latest snapshot, which is how the old coordinator behaves. - return runtime.scheduleReadOperation( + return runtime.scheduleWriteOperation( "classic-group-heartbeat", topicPartitionFor(request.groupId()), - (coordinator, __) -> coordinator.classicGroupHeartbeat(context, request) + Duration.ofMillis(config.offsetCommitTimeoutMs), + coordinator -> coordinator.classicGroupHeartbeat(context, request) ).exceptionally(exception -> handleOperationException( "classic-group-heartbeat", request, diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java index c01605926ba71..081c9764944cd 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java @@ -355,9 +355,10 @@ public CoordinatorResult classicGroupSync( * @param context The request context. * @param request The actual Heartbeat request. * - * @return The HeartbeatResponse. + * @return A Result containing the heartbeat response and + * a list of records to update the state machine. */ - public HeartbeatResponseData classicGroupHeartbeat( + public CoordinatorResult classicGroupHeartbeat( RequestContext context, HeartbeatRequestData request ) { diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index 5520676d21b1d..b912cb6ac36d0 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -58,7 +58,7 @@ import org.apache.kafka.common.requests.RequestContext; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; +import org.apache.kafka.coordinator.group.assignor.ConsumerGroupPartitionAssignor; import org.apache.kafka.coordinator.group.assignor.PartitionAssignorException; import org.apache.kafka.coordinator.group.assignor.SubscriptionType; import org.apache.kafka.coordinator.group.consumer.Assignment; @@ -158,7 +158,7 @@ public static class Builder { private SnapshotRegistry snapshotRegistry = null; private Time time = null; private CoordinatorTimer timer = null; - private List consumerGroupAssignors = null; + private List consumerGroupAssignors = null; private int consumerGroupMaxSize = Integer.MAX_VALUE; private int consumerGroupHeartbeatIntervalMs = 5000; private int consumerGroupMetadataRefreshIntervalMs = Integer.MAX_VALUE; @@ -192,7 +192,7 @@ Builder withTimer(CoordinatorTimer timer) { return this; } - Builder withConsumerGroupAssignors(List consumerGroupAssignors) { + Builder withConsumerGroupAssignors(List consumerGroupAssignors) { this.consumerGroupAssignors = consumerGroupAssignors; return this; } @@ -323,14 +323,14 @@ GroupMetadataManager build() { private final GroupCoordinatorMetricsShard metrics; /** - * The supported partition assignors keyed by their name. + * The supported consumer group partition assignors keyed by their name. */ - private final Map assignors; + private final Map consumerGroupAssignors; /** - * The default assignor used. + * The default consumer group assignor used. */ - private final PartitionAssignor defaultAssignor; + private final ConsumerGroupPartitionAssignor defaultConsumerGroupAssignor; /** * The classic and consumer groups keyed by their name. @@ -412,7 +412,7 @@ private GroupMetadataManager( Time time, CoordinatorTimer timer, GroupCoordinatorMetricsShard metrics, - List assignors, + List consumerGroupAssignors, MetadataImage metadataImage, int consumerGroupMaxSize, int consumerGroupSessionTimeoutMs, @@ -432,8 +432,8 @@ private GroupMetadataManager( this.timer = timer; this.metrics = metrics; this.metadataImage = metadataImage; - this.assignors = assignors.stream().collect(Collectors.toMap(PartitionAssignor::name, Function.identity())); - this.defaultAssignor = assignors.get(0); + this.consumerGroupAssignors = consumerGroupAssignors.stream().collect(Collectors.toMap(ConsumerGroupPartitionAssignor::name, Function.identity())); + this.defaultConsumerGroupAssignor = consumerGroupAssignors.get(0); this.groups = new TimelineHashMap<>(snapshotRegistry, 0); this.groupsByTopics = new TimelineHashMap<>(snapshotRegistry, 0); this.consumerGroupMaxSize = consumerGroupMaxSize; @@ -539,7 +539,7 @@ public List consumerGroupDescr try { describedGroups.add(consumerGroup(groupId, committedOffset).asDescribedGroup( committedOffset, - defaultAssignor.name(), + defaultConsumerGroupAssignor.name(), metadataImage.topics() )); } catch (GroupIdNotFoundException exception) { @@ -1041,9 +1041,9 @@ private void throwIfConsumerGroupHeartbeatRequestIsInvalid( throw new InvalidRequestException("MemberEpoch is invalid."); } - if (request.serverAssignor() != null && !assignors.containsKey(request.serverAssignor())) { + if (request.serverAssignor() != null && !consumerGroupAssignors.containsKey(request.serverAssignor())) { throw new UnsupportedAssignorException("ServerAssignor " + request.serverAssignor() - + " is not supported. Supported assignors: " + String.join(", ", assignors.keySet()) + + " is not supported. Supported assignors: " + String.join(", ", consumerGroupAssignors.keySet()) + "."); } } @@ -1273,6 +1273,7 @@ private void throwIfRebalanceInProgress( // member has already rejoined, so it needs to first finish revoking the partitions and the reconciliation, // and then the next rejoin will be triggered automatically if needed. if (group.groupEpoch() > member.memberEpoch() && !member.state().equals(MemberState.UNREVOKED_PARTITIONS)) { + scheduleConsumerGroupJoinTimeoutIfAbsent(group.groupId(), member.memberId(), member.rebalanceTimeoutMs()); throw Errors.REBALANCE_IN_PROGRESS.exception( String.format("A new rebalance is triggered in group %s and member %s should rejoin to catch up.", group.groupId(), member.memberId()) @@ -1753,6 +1754,7 @@ private CoordinatorResult classicGroupJoinToConsumerGro CompletableFuture appendFuture = new CompletableFuture<>(); appendFuture.whenComplete((__, t) -> { if (t == null) { + cancelConsumerGroupJoinTimeout(groupId, response.memberId()); scheduleConsumerGroupSessionTimeout(groupId, response.memberId(), sessionTimeoutMs); // The sync timeout ensures that the member send sync request within the rebalance timeout. scheduleConsumerGroupSyncTimeout(groupId, response.memberId(), request.rebalanceTimeoutMs()); @@ -1890,15 +1892,17 @@ private Assignment updateTargetAssignment( String preferredServerAssignor = group.computePreferredServerAssignor( member, updatedMember - ).orElse(defaultAssignor.name()); + ).orElse(defaultConsumerGroupAssignor.name()); try { TargetAssignmentBuilder assignmentResultBuilder = - new TargetAssignmentBuilder(group.groupId(), groupEpoch, assignors.get(preferredServerAssignor)) + new TargetAssignmentBuilder(group.groupId(), groupEpoch, consumerGroupAssignors.get(preferredServerAssignor)) .withMembers(group.members()) .withStaticMembers(group.staticMembers()) .withSubscriptionMetadata(subscriptionMetadata) .withSubscriptionType(subscriptionType) .withTargetAssignment(group.targetAssignment()) + .withInvertedTargetAssignment(group.invertedTargetAssignment()) + .withTopicsImage(metadataImage.topics()) .addOrUpdateMember(updatedMember.memberId(), updatedMember); TargetAssignmentBuilder.TargetAssignmentResult assignmentResult; // A new static member is replacing an older one with the same subscriptions. @@ -2061,6 +2065,7 @@ private void removeMember(List records, String groupId, Strin private void cancelTimers(String groupId, String memberId) { cancelConsumerGroupSessionTimeout(groupId, memberId); cancelConsumerGroupRebalanceTimeout(groupId, memberId); + cancelConsumerGroupJoinTimeout(groupId, memberId); cancelConsumerGroupSyncTimeout(groupId, memberId); } @@ -2077,6 +2082,39 @@ private void scheduleConsumerGroupSessionTimeout( scheduleConsumerGroupSessionTimeout(groupId, memberId, consumerGroupSessionTimeoutMs); } + /** + * Fences a member from a consumer group. Returns an empty CoordinatorResult + * if the group or the member doesn't exist. + * + * @param groupId The group id. + * @param memberId The member id. + * @param reason The reason for fencing the member. + * + * @return The CoordinatorResult to be applied. + */ + private CoordinatorResult consumerGroupFenceMemberOperation( + String groupId, + String memberId, + String reason + ) { + try { + ConsumerGroup group = consumerGroup(groupId); + ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false); + log.info("[GroupId {}] Member {} fenced from the group because {}.", + groupId, memberId, reason); + + return consumerGroupFenceMember(group, member, null); + } catch (GroupIdNotFoundException ex) { + log.debug("[GroupId {}] Could not fence {} because the group does not exist.", + groupId, memberId); + } catch (UnknownMemberIdException ex) { + log.debug("[GroupId {}] Could not fence {} because the member does not exist.", + groupId, memberId); + } + + return new CoordinatorResult<>(Collections.emptyList()); + } + /** * Schedules (or reschedules) the session timeout for the member. * @@ -2089,25 +2127,13 @@ private void scheduleConsumerGroupSessionTimeout( String memberId, int sessionTimeoutMs ) { - String key = consumerGroupSessionTimeoutKey(groupId, memberId); - timer.schedule(key, sessionTimeoutMs, TimeUnit.MILLISECONDS, true, () -> { - try { - ConsumerGroup group = consumerGroup(groupId); - ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false); - log.info("[GroupId {}] Member {} fenced from the group because its session expired.", - groupId, memberId); - - return consumerGroupFenceMember(group, member, null); - } catch (GroupIdNotFoundException ex) { - log.debug("[GroupId {}] Could not fence {} because the group does not exist.", - groupId, memberId); - } catch (UnknownMemberIdException ex) { - log.debug("[GroupId {}] Could not fence {} because the member does not exist.", - groupId, memberId); - } - - return new CoordinatorResult<>(Collections.emptyList()); - }); + timer.schedule( + consumerGroupSessionTimeoutKey(groupId, memberId), + sessionTimeoutMs, + TimeUnit.MILLISECONDS, + true, + () -> consumerGroupFenceMemberOperation(groupId, memberId, "the member session expired.") + ); } /** @@ -2180,36 +2206,58 @@ private void cancelConsumerGroupRebalanceTimeout( } /** - * Schedules a sync timeout for the member. + * Schedules a join timeout for the member if there's not a join timeout. * * @param groupId The group id. * @param memberId The member id. * @param rebalanceTimeoutMs The rebalance timeout. */ - private void scheduleConsumerGroupSyncTimeout( + private void scheduleConsumerGroupJoinTimeoutIfAbsent( String groupId, String memberId, int rebalanceTimeoutMs ) { - String key = consumerGroupSyncKey(groupId, memberId); - timer.schedule(key, rebalanceTimeoutMs, TimeUnit.MILLISECONDS, true, () -> { - try { - ConsumerGroup group = consumerGroup(groupId); - ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false); - log.info("[GroupId {}] Member {} fenced from the group because its session expired.", - groupId, memberId); + timer.scheduleIfAbsent( + consumerGroupJoinKey(groupId, memberId), + rebalanceTimeoutMs, + TimeUnit.MILLISECONDS, + true, + () -> consumerGroupFenceMemberOperation(groupId, memberId, "the classic member failed to join within the rebalance timeout.") + ); + } - return consumerGroupFenceMember(group, member, null); - } catch (GroupIdNotFoundException ex) { - log.debug("[GroupId {}] Could not fence {} because the group does not exist.", - groupId, memberId); - } catch (UnknownMemberIdException ex) { - log.debug("[GroupId {}] Could not fence {} because the member does not exist.", - groupId, memberId); - } + /** + * Cancels the join timeout of the member. + * + * @param groupId The group id. + * @param memberId The member id. + */ + private void cancelConsumerGroupJoinTimeout( + String groupId, + String memberId + ) { + timer.cancel(consumerGroupJoinKey(groupId, memberId)); + } - return new CoordinatorResult<>(Collections.emptyList()); - }); + /** + * Schedules a sync timeout for the member. + * + * @param groupId The group id. + * @param memberId The member id. + * @param rebalanceTimeoutMs The rebalance timeout. + */ + private void scheduleConsumerGroupSyncTimeout( + String groupId, + String memberId, + int rebalanceTimeoutMs + ) { + timer.schedule( + consumerGroupSyncKey(groupId, memberId), + rebalanceTimeoutMs, + TimeUnit.MILLISECONDS, + true, + () -> consumerGroupFenceMemberOperation(groupId, memberId, "the member failed to sync within timeout.") + ); } /** @@ -3938,10 +3986,17 @@ public CoordinatorResult classicGroupSync( RequestContext context, SyncGroupRequestData request, CompletableFuture responseFuture - ) throws UnknownMemberIdException, GroupIdNotFoundException { - Group group = groups.get(request.groupId(), Long.MAX_VALUE); + ) throws UnknownMemberIdException { + Group group; + try { + group = group(request.groupId()); + } catch (GroupIdNotFoundException e) { + responseFuture.complete(new SyncGroupResponseData() + .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())); + return EMPTY_RESULT; + } - if (group == null || group.isEmpty()) { + if (group.isEmpty()) { responseFuture.complete(new SyncGroupResponseData() .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())); return EMPTY_RESULT; @@ -4072,19 +4127,7 @@ private CoordinatorResult classicGroupSyncToConsumerGro String groupId = request.groupId(); String memberId = request.memberId(); String instanceId = request.groupInstanceId(); - - ConsumerGroupMember member; - if (instanceId == null) { - member = group.getOrMaybeCreateMember(request.memberId(), false); - } else { - member = group.staticMember(instanceId); - if (member == null) { - throw new UnknownMemberIdException( - String.format("Member with instance id %s is not a member of group %s.", instanceId, groupId) - ); - } - throwIfInstanceIdIsFenced(member, groupId, memberId, instanceId); - } + ConsumerGroupMember member = validateConsumerGroupMember(group, memberId, instanceId); throwIfMemberDoesNotUseClassicProtocol(member); throwIfGenerationIdUnmatched(member.memberId(), member.memberEpoch(), request.generationId()); @@ -4209,23 +4252,57 @@ private void removePendingSyncMember( * @param context The request context. * @param request The actual Heartbeat request. * - * @return The Heartbeat response. + * @return The coordinator result that contains the heartbeat response. */ - public HeartbeatResponseData classicGroupHeartbeat( + public CoordinatorResult classicGroupHeartbeat( RequestContext context, HeartbeatRequestData request ) { - ClassicGroup group = getOrMaybeCreateClassicGroup(request.groupId(), false); + Group group; + try { + group = group(request.groupId()); + } catch (GroupIdNotFoundException e) { + throw new UnknownMemberIdException( + String.format("Group %s not found.", request.groupId()) + ); + } + if (group.type() == CLASSIC) { + return classicGroupHeartbeatToClassicGroup((ClassicGroup) group, context, request); + } else { + return classicGroupHeartbeatToConsumerGroup((ConsumerGroup) group, context, request); + } + } + + /** + * Handle a classic group HeartbeatRequest to a classic group. + * + * @param group The ClassicGroup. + * @param context The request context. + * @param request The actual Heartbeat request. + * + * @return The coordinator result that contains the heartbeat response. + */ + private CoordinatorResult classicGroupHeartbeatToClassicGroup( + ClassicGroup group, + RequestContext context, + HeartbeatRequestData request + ) { validateClassicGroupHeartbeat(group, request.memberId(), request.groupInstanceId(), request.generationId()); switch (group.currentState()) { case EMPTY: - return new HeartbeatResponseData().setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()); + return new CoordinatorResult<>( + Collections.emptyList(), + new HeartbeatResponseData().setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()) + ); case PREPARING_REBALANCE: rescheduleClassicGroupMemberHeartbeat(group, group.member(request.memberId())); - return new HeartbeatResponseData().setErrorCode(Errors.REBALANCE_IN_PROGRESS.code()); + return new CoordinatorResult<>( + Collections.emptyList(), + new HeartbeatResponseData().setErrorCode(Errors.REBALANCE_IN_PROGRESS.code()) + ); case COMPLETING_REBALANCE: case STABLE: @@ -4233,7 +4310,10 @@ public HeartbeatResponseData classicGroupHeartbeat( // is in CompletingRebalance state. In this case, we should treat them as // normal heartbeat requests and reset the timer rescheduleClassicGroupMemberHeartbeat(group, group.member(request.memberId())); - return new HeartbeatResponseData(); + return new CoordinatorResult<>( + Collections.emptyList(), + new HeartbeatResponseData() + ); default: throw new IllegalStateException("Reached unexpected state " + @@ -4274,20 +4354,210 @@ private void validateClassicGroupHeartbeat( } } + /** + * Handle a classic group HeartbeatRequest to a consumer group. A response with + * REBALANCE_IN_PROGRESS is returned if 1) the member epoch is smaller than the + * group epoch, 2) the member is in UNREVOKED_PARTITIONS, or 3) the member is in + * UNRELEASED_PARTITIONS and all its partitions pending assignment are free. + * + * @param group The ConsumerGroup. + * @param context The request context. + * @param request The actual Heartbeat request. + * + * @return The coordinator result that contains the heartbeat response. + */ + private CoordinatorResult classicGroupHeartbeatToConsumerGroup( + ConsumerGroup group, + RequestContext context, + HeartbeatRequestData request + ) throws UnknownMemberIdException, FencedInstanceIdException, IllegalGenerationException { + String groupId = request.groupId(); + String memberId = request.memberId(); + String instanceId = request.groupInstanceId(); + ConsumerGroupMember member = validateConsumerGroupMember(group, memberId, instanceId); + + throwIfMemberDoesNotUseClassicProtocol(member); + throwIfGenerationIdUnmatched(memberId, member.memberEpoch(), request.generationId()); + + scheduleConsumerGroupSessionTimeout(groupId, memberId, member.classicProtocolSessionTimeout().get()); + + Errors error = Errors.NONE; + // The member should rejoin if any of the following conditions is met. + // 1) The group epoch is bumped so the member need to rejoin to catch up. + // 2) The member needs to revoke some partitions and rejoin to reconcile with the new epoch. + // 3) The member's partitions pending assignment are free, so it can rejoin to get the complete assignment. + if (member.memberEpoch() < group.groupEpoch() || + member.state() == MemberState.UNREVOKED_PARTITIONS || + (member.state() == MemberState.UNRELEASED_PARTITIONS && !group.waitingOnUnreleasedPartition(member))) { + error = Errors.REBALANCE_IN_PROGRESS; + scheduleConsumerGroupJoinTimeoutIfAbsent(groupId, memberId, member.rebalanceTimeoutMs()); + } + + return new CoordinatorResult<>( + Collections.emptyList(), + new HeartbeatResponseData().setErrorCode(error.code()) + ); + } + + /** + * Validates that (1) the instance id exists and is mapped to the member id + * if the group instance id is provided; and (2) the member id exists in the group. + * + * @param group The consumer group. + * @param memberId The member id. + * @param instanceId The instance id. + * + * @return The ConsumerGroupMember. + */ + private ConsumerGroupMember validateConsumerGroupMember( + ConsumerGroup group, + String memberId, + String instanceId + ) throws UnknownMemberIdException, FencedInstanceIdException { + ConsumerGroupMember member; + if (instanceId == null) { + member = group.getOrMaybeCreateMember(memberId, false); + } else { + member = group.staticMember(instanceId); + if (member == null) { + throw new UnknownMemberIdException( + String.format("Member with instance id %s is not a member of group %s.", instanceId, group.groupId()) + ); + } + throwIfInstanceIdIsFenced(member, group.groupId(), memberId, instanceId); + } + return member; + } + /** * Handle a classic LeaveGroupRequest. * * @param context The request context. * @param request The actual LeaveGroup request. * + * @return The LeaveGroup response and the records to append. + */ + public CoordinatorResult classicGroupLeave( + RequestContext context, + LeaveGroupRequestData request + ) throws UnknownMemberIdException { + Group group; + try { + group = group(request.groupId()); + } catch (GroupIdNotFoundException e) { + throw new UnknownMemberIdException(String.format("Group %s not found.", request.groupId())); + } + + if (group.type() == CLASSIC) { + return classicGroupLeaveToClassicGroup((ClassicGroup) group, context, request); + } else { + return classicGroupLeaveToConsumerGroup((ConsumerGroup) group, context, request); + } + } + + /** + * Handle a classic LeaveGroupRequest to a ConsumerGroup. + * + * @param group The ConsumerGroup. + * @param context The request context. + * @param request The actual LeaveGroup request. + * + * @return The LeaveGroup response and the records to append. + */ + private CoordinatorResult classicGroupLeaveToConsumerGroup( + ConsumerGroup group, + RequestContext context, + LeaveGroupRequestData request + ) throws UnknownMemberIdException { + String groupId = group.groupId(); + List memberResponses = new ArrayList<>(); + Set validLeaveGroupMembers = new HashSet<>(); + List records = new ArrayList<>(); + + for (MemberIdentity memberIdentity : request.members()) { + String memberId = memberIdentity.memberId(); + String instanceId = memberIdentity.groupInstanceId(); + String reason = memberIdentity.reason() != null ? memberIdentity.reason() : "not provided"; + + ConsumerGroupMember member; + try { + if (instanceId == null) { + member = group.getOrMaybeCreateMember(memberId, false); + throwIfMemberDoesNotUseClassicProtocol(member); + + log.info("[Group {}] Dynamic Member {} has left group " + + "through explicit `LeaveGroup` request; client reason: {}", + groupId, memberId, reason); + } else { + member = group.staticMember(instanceId); + throwIfStaticMemberIsUnknown(member, instanceId); + // The LeaveGroup API allows administrative removal of members by GroupInstanceId + // in which case we expect the MemberId to be undefined. + if (!UNKNOWN_MEMBER_ID.equals(memberId)) { + throwIfInstanceIdIsFenced(member, groupId, memberId, instanceId); + } + throwIfMemberDoesNotUseClassicProtocol(member); + + memberId = member.memberId(); + log.info("[Group {}] Static Member {} with instance id {} has left group " + + "through explicit `LeaveGroup` request; client reason: {}", + groupId, memberId, instanceId, reason); + } + + removeMember(records, groupId, memberId); + cancelTimers(groupId, memberId); + memberResponses.add( + new MemberResponse() + .setMemberId(memberId) + .setGroupInstanceId(instanceId) + ); + validLeaveGroupMembers.add(member); + } catch (KafkaException e) { + memberResponses.add( + new MemberResponse() + .setMemberId(memberId) + .setGroupInstanceId(instanceId) + .setErrorCode(Errors.forException(e).code()) + ); + } + } + + if (!records.isEmpty()) { + // Maybe update the subscription metadata. + Map subscriptionMetadata = group.computeSubscriptionMetadata( + group.computeSubscribedTopicNames(validLeaveGroupMembers), + metadataImage.topics(), + metadataImage.cluster() + ); + + if (!subscriptionMetadata.equals(group.subscriptionMetadata())) { + log.info("[GroupId {}] Computed new subscription metadata: {}.", + group.groupId(), subscriptionMetadata); + records.add(newGroupSubscriptionMetadataRecord(group.groupId(), subscriptionMetadata)); + } + + // Bump the group epoch. + records.add(newGroupEpochRecord(groupId, group.groupEpoch() + 1)); + } + + return new CoordinatorResult<>(records, new LeaveGroupResponseData().setMembers(memberResponses)); + } + + /** + * Handle a classic LeaveGroupRequest to a ClassicGroup. + * + * @param group The ClassicGroup. + * @param context The request context. + * @param request The actual LeaveGroup request. + * * @return The LeaveGroup response and the GroupMetadata record to append if the group * no longer has any members. */ - public CoordinatorResult classicGroupLeave( + private CoordinatorResult classicGroupLeaveToClassicGroup( + ClassicGroup group, RequestContext context, LeaveGroupRequestData request - ) throws UnknownMemberIdException, GroupIdNotFoundException { - ClassicGroup group = getOrMaybeCreateClassicGroup(request.groupId(), false); + ) throws UnknownMemberIdException { if (group.isInState(DEAD)) { return new CoordinatorResult<>( Collections.emptyList(), @@ -4583,6 +4853,20 @@ static String classicGroupSyncKey(String groupId) { return "sync-" + groupId; } + /** + * Generate a consumer group join key for the timer. + * + * Package private for testing. + * + * @param groupId The group id. + * @param memberId The member id. + * + * @return the sync key. + */ + static String consumerGroupJoinKey(String groupId, String memberId) { + return "join-" + groupId + "-" + memberId; + } + /** * Generate a consumer group sync key for the timer. * diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AssignmentMemberSpec.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AssignmentMemberSpec.java index 4a93fccae35d5..2a91a111aa27b 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AssignmentMemberSpec.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AssignmentMemberSpec.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.Uuid; -import java.util.Collection; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -41,7 +40,7 @@ public class AssignmentMemberSpec { /** * Topics Ids that the member is subscribed to. */ - private final Collection subscribedTopicIds; + private final Set subscribedTopicIds; /** * Partitions assigned keyed by topicId. @@ -63,9 +62,9 @@ public Optional rackId() { } /** - * @return Collection of subscribed topic Ids. + * @return Set of subscribed topic Ids. */ - public Collection subscribedTopicIds() { + public Set subscribedTopicIds() { return subscribedTopicIds; } @@ -79,7 +78,7 @@ public Map> assignedPartitions() { public AssignmentMemberSpec( Optional instanceId, Optional rackId, - Collection subscribedTopicIds, + Set subscribedTopicIds, Map> assignedPartitions ) { Objects.requireNonNull(instanceId); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ConsumerGroupPartitionAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ConsumerGroupPartitionAssignor.java new file mode 100644 index 0000000000000..da315b38d6c39 --- /dev/null +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ConsumerGroupPartitionAssignor.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.group.assignor; + +import org.apache.kafka.common.annotation.InterfaceStability; + +/** + * Server-side partition assignor for consumer groups used by the GroupCoordinator. + * + * The interface is kept in an internal module until KIP-848 is fully + * implemented and ready to be released. + */ +@InterfaceStability.Unstable +public interface ConsumerGroupPartitionAssignor extends PartitionAssignor { +} diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilder.java index f8d165e1bd124..d6f83df00572c 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilder.java @@ -107,8 +107,8 @@ public class GeneralUniformAssignmentBuilder extends AbstractUniformAssignmentBu */ private final PartitionMovements partitionMovements; - public GeneralUniformAssignmentBuilder(AssignmentSpec assignmentSpec, SubscribedTopicDescriber subscribedTopicDescriber) { - this.members = assignmentSpec.members(); + public GeneralUniformAssignmentBuilder(GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber) { + this.members = groupSpec.members(); this.subscribedTopicDescriber = subscribedTopicDescriber; this.subscribedTopicIds = new HashSet<>(); this.membersPerTopic = new HashMap<>(); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpec.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpec.java new file mode 100644 index 0000000000000..296dedb52902b --- /dev/null +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpec.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.group.assignor; + +import org.apache.kafka.common.Uuid; + +import java.util.Map; + +/** + * The group metadata specifications required to compute the target assignment. + */ +public interface GroupSpec { + /** + * @return Member metadata keyed by member Id. + */ + Map members(); + + /** + * @return The group's subscription type. + */ + SubscriptionType subscriptionType(); + + /** + * @return True, if the partition is currently assigned to a member. + * False, otherwise. + */ + boolean isPartitionAssigned(Uuid topicId, int partitionId); +} diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AssignmentSpec.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImpl.java similarity index 55% rename from group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AssignmentSpec.java rename to group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImpl.java index ec65dc6decc7d..0194727c7dcf2 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AssignmentSpec.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImpl.java @@ -16,13 +16,15 @@ */ package org.apache.kafka.coordinator.group.assignor; +import org.apache.kafka.common.Uuid; + import java.util.Map; import java.util.Objects; /** * The assignment specification for a consumer group. */ -public class AssignmentSpec { +public class GroupSpecImpl implements GroupSpec { /** * The member metadata keyed by member Id. */ @@ -33,44 +35,76 @@ public class AssignmentSpec { */ private final SubscriptionType subscriptionType; - public AssignmentSpec( + /** + * Reverse lookup map representing topic partitions with + * their current member assignments. + */ + private final Map> invertedTargetAssignment; + + public GroupSpecImpl( Map members, - SubscriptionType subscriptionType + SubscriptionType subscriptionType, + Map> invertedTargetAssignment ) { Objects.requireNonNull(members); + Objects.requireNonNull(subscriptionType); + Objects.requireNonNull(invertedTargetAssignment); this.members = members; this.subscriptionType = subscriptionType; + this.invertedTargetAssignment = invertedTargetAssignment; } /** - * @return Member metadata keyed by member Id. + * {@inheritDoc} */ + @Override public Map members() { return members; } /** - * @return The group's subscription type. + * {@inheritDoc} */ + @Override public SubscriptionType subscriptionType() { return subscriptionType; } + /** + * {@inheritDoc} + */ + @Override + public boolean isPartitionAssigned(Uuid topicId, int partitionId) { + Map partitionMap = invertedTargetAssignment.get(topicId); + if (partitionMap == null) { + return false; + } + return partitionMap.containsKey(partitionId); + } + @Override public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - AssignmentSpec that = (AssignmentSpec) o; + GroupSpecImpl that = (GroupSpecImpl) o; return subscriptionType == that.subscriptionType && - members.equals(that.members); + members.equals(that.members) && + invertedTargetAssignment.equals(that.invertedTargetAssignment); } @Override public int hashCode() { - return Objects.hash(members, subscriptionType); + int result = members.hashCode(); + result = 31 * result + subscriptionType.hashCode(); + result = 31 * result + invertedTargetAssignment.hashCode(); + return result; } + @Override public String toString() { - return "AssignmentSpec(members=" + members + ", subscriptionType=" + subscriptionType.toString() + ')'; + return "GroupSpecImpl(members=" + members + + ", subscriptionType=" + subscriptionType + + ", invertedTargetAssignment=" + invertedTargetAssignment + + ')'; } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java index a14363708e37c..34e825665204c 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java @@ -18,29 +18,18 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.server.common.TopicIdPartition; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Queue; import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -import static java.lang.Math.min; /** - * The optimized uniform assignment builder is used to generate the target assignment for a consumer group with + * The homogenous uniform assignment builder is used to generate the target assignment for a consumer group with * all its members subscribed to the same set of topics. - * It is optimized since the assignment can be done in fewer, less complicated steps compared to when - * the subscriptions are different across the members. * * Assignments are done according to the following principles: * @@ -53,13 +42,22 @@ * The assignment builder prioritizes the properties in the following order: * Balance > Stickiness. */ -public class OptimizedUniformAssignmentBuilder extends AbstractUniformAssignmentBuilder { - private static final Logger LOG = LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class); +public class OptimizedUniformAssignmentBuilder { + private static final Class UNMODIFIABLE_MAP_CLASS = Collections.unmodifiableMap(new HashMap<>()).getClass(); + private static final Class EMPTY_MAP_CLASS = Collections.emptyMap().getClass(); + + /** + * @return True if the provided map is an UnmodifiableMap or EmptyMap. Those classes are not + * public hence we cannot use the `instanceof` operator. + */ + private static boolean isImmutableMap(Map map) { + return UNMODIFIABLE_MAP_CLASS.isInstance(map) || EMPTY_MAP_CLASS.isInstance(map); + } /** * The assignment specification which includes member metadata. */ - private final AssignmentSpec assignmentSpec; + private final GroupSpec groupSpec; /** * The topic and partition metadata describer. @@ -72,58 +70,53 @@ public class OptimizedUniformAssignmentBuilder extends AbstractUniformAssignment private final Set subscribedTopicIds; /** - * The number of members to receive an extra partition beyond the minimum quota. - * Minimum Quota = Total Partitions / Total Members - * Example: If there are 11 partitions to be distributed among 3 members, - * each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 3) members get an extra partition. + * The members that are below their quota. */ - private int remainingMembersToGetAnExtraPartition; - - /** - * Members mapped to the remaining number of partitions needed to meet the minimum quota. - * Minimum quota = total partitions / total members. - */ - private Map potentiallyUnfilledMembers; + private final List unfilledMembers; /** * The partitions that still need to be assigned. * Initially this contains all the subscribed topics' partitions. */ - private Set unassignedPartitions; + private final List unassignedPartitions; /** * The target assignment. */ private final Map targetAssignment; - OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, SubscribedTopicDescriber subscribedTopicDescriber) { - this.assignmentSpec = assignmentSpec; + /** + * The minimum number of partitions that a member must have. + * Minimum quota = total partitions / total members. + */ + private int minimumMemberQuota; + + /** + * The number of members to receive an extra partition beyond the minimum quota. + * Example: If there are 11 partitions to be distributed among 3 members, + * each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 3) members get an extra partition. + */ + private int remainingMembersToGetAnExtraPartition; + + OptimizedUniformAssignmentBuilder(GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber) { + this.groupSpec = groupSpec; this.subscribedTopicDescriber = subscribedTopicDescriber; - this.subscribedTopicIds = new HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds()); - this.potentiallyUnfilledMembers = new HashMap<>(); + this.subscribedTopicIds = new HashSet<>(groupSpec.members().values().iterator().next().subscribedTopicIds()); + this.unfilledMembers = new ArrayList<>(); + this.unassignedPartitions = new ArrayList<>(); this.targetAssignment = new HashMap<>(); } /** - * Here's the step-by-step breakdown of the assignment process: - * - *
    • Compute the quotas of partitions for each member based on the total partitions and member count.
    • - *
    • Initialize unassigned partitions to all the topic partitions and - * remove partitions from the list as and when they are assigned.
    • - *
    • For existing assignments, retain partitions based on the determined quota.
    • - *
    • Identify members that haven't fulfilled their partition quota or are eligible to receive extra partitions.
    • - *
    • Proceed with a round-robin assignment according to quotas. - * For each unassigned partition, locate the first compatible member from the potentially unfilled list.
    • + * Compute the new assignment for the group. */ - @Override - protected GroupAssignment buildAssignment() throws PartitionAssignorException { - int totalPartitionsCount = 0; - + public GroupAssignment build() throws PartitionAssignorException { if (subscribedTopicIds.isEmpty()) { - LOG.debug("The subscription list is empty, returning an empty assignment"); return new GroupAssignment(Collections.emptyMap()); } + // Compute the list of unassigned partitions. + int totalPartitionsCount = 0; for (Uuid topicId : subscribedTopicIds) { int partitionCount = subscribedTopicDescriber.numPartitions(topicId); if (partitionCount == -1) { @@ -131,213 +124,158 @@ protected GroupAssignment buildAssignment() throws PartitionAssignorException { "Members are subscribed to topic " + topicId + " which doesn't exist in the topic metadata." ); } else { + for (int i = 0; i < partitionCount; i++) { + if (!groupSpec.isPartitionAssigned(topicId, i)) { + unassignedPartitions.add(new TopicIdPartition(topicId, i)); + } + } totalPartitionsCount += partitionCount; } } - // The minimum required quota that each member needs to meet for a balanced assignment. - // This is the same for all members. - final int numberOfMembers = assignmentSpec.members().size(); - final int minQuota = totalPartitionsCount / numberOfMembers; + // Compute the minimum required quota per member and the number of members + // that should receive an extra partition. + int numberOfMembers = groupSpec.members().size(); + minimumMemberQuota = totalPartitionsCount / numberOfMembers; remainingMembersToGetAnExtraPartition = totalPartitionsCount % numberOfMembers; - assignmentSpec.members().keySet().forEach(memberId -> - targetAssignment.put(memberId, new MemberAssignment(new HashMap<>()) - )); - - unassignedPartitions = topicIdPartitions(subscribedTopicIds, subscribedTopicDescriber); - potentiallyUnfilledMembers = assignStickyPartitions(minQuota); + // Revoke the partitions that either are not part of the member's subscriptions or + // exceed the maximum quota assigned to each member. + maybeRevokePartitions(); - unassignedPartitionsRoundRobinAssignment(); - - if (!unassignedPartitions.isEmpty()) { - throw new PartitionAssignorException("Partitions were left unassigned"); - } + // Assign the unassigned partitions to the members with space. + assignRemainingPartitions(); return new GroupAssignment(targetAssignment); } /** - * Retains a set of partitions from the existing assignment and includes them in the target assignment. - * Only relevant partitions that exist in the current topic metadata and subscriptions are considered. - * - *

      For each member: - *

        - *
      1. Find the valid current assignment considering topic subscriptions and metadata
      2. - *
      3. If the current assignment exists, retain partitions up to the minimum quota.
      4. - *
      5. If the current assignment size is greater than the minimum quota and - * there are members that could get an extra partition, assign the next partition as well.
      6. - *
      7. Finally, if the member's current assignment size is less than the minimum quota, - * add them to the potentially unfilled members map and track the number of remaining - * partitions required to meet the quota.
      8. - *
      - *

      + * Revoke the partitions that either are not part of the member's subscriptions or + * exceed the maximum quota assigned to each member. * - * @return Members mapped to the remaining number of partitions needed to meet the minimum quota, - * including members that are eligible to receive an extra partition. + * This method ensures that the original assignment is not copied if it is not + * altered. */ - private Map assignStickyPartitions(int minQuota) { - Map potentiallyUnfilledMembers = new HashMap<>(); - - assignmentSpec.members().forEach((memberId, assignmentMemberSpec) -> { - List validCurrentMemberAssignment = validCurrentMemberAssignment( - assignmentMemberSpec.assignedPartitions() - ); - - int currentAssignmentSize = validCurrentMemberAssignment.size(); - // Number of partitions required to meet the minimum quota. - int remaining = minQuota - currentAssignmentSize; - - if (currentAssignmentSize > 0) { - int retainedPartitionsCount = min(currentAssignmentSize, minQuota); - IntStream.range(0, retainedPartitionsCount).forEach(i -> { - TopicIdPartition topicIdPartition = validCurrentMemberAssignment.get(i); - addPartitionToAssignment( - targetAssignment, - memberId, - topicIdPartition.topicId(), - topicIdPartition.partitionId() - ); - unassignedPartitions.remove(topicIdPartition); - }); - - // The extra partition is located at the last index from the previous step. - if (remaining < 0 && remainingMembersToGetAnExtraPartition > 0) { - TopicIdPartition topicIdPartition = validCurrentMemberAssignment.get(retainedPartitionsCount); - addPartitionToAssignment( - targetAssignment, - memberId, - topicIdPartition.topicId(), - topicIdPartition.partitionId() - ); - unassignedPartitions.remove(topicIdPartition); - remainingMembersToGetAnExtraPartition--; - } + private void maybeRevokePartitions() { + for (Map.Entry entry : groupSpec.members().entrySet()) { + String memberId = entry.getKey(); + AssignmentMemberSpec assignmentMemberSpec = entry.getValue(); + Map> oldAssignment = assignmentMemberSpec.assignedPartitions(); + Map> newAssignment = null; + + // The assignor expects to receive the assignment as an immutable map. It leverages + // this knowledge in order to avoid having to copy all assignments. + if (!isImmutableMap(oldAssignment)) { + throw new IllegalStateException("The assignor expect an immutable map."); } - if (remaining >= 0) { - potentiallyUnfilledMembers.put(memberId, remaining); + int quota = minimumMemberQuota; + if (remainingMembersToGetAnExtraPartition > 0) { + quota++; + remainingMembersToGetAnExtraPartition--; } - }); - return potentiallyUnfilledMembers; - } + for (Map.Entry> topicPartitions : oldAssignment.entrySet()) { + Uuid topicId = topicPartitions.getKey(); + Set partitions = topicPartitions.getValue(); + + if (subscribedTopicIds.contains(topicId)) { + if (partitions.size() <= quota) { + quota -= partitions.size(); + } else { + for (Integer partition : partitions) { + if (quota > 0) { + quota--; + } else { + if (newAssignment == null) { + // If the new assignment is null, we create a deep copy of the + // original assignment so that we can alter it. + newAssignment = deepCopy(oldAssignment); + } + // Remove the partition from the new assignment. + Set parts = newAssignment.get(topicId); + parts.remove(partition); + if (parts.isEmpty()) { + newAssignment.remove(topicId); + } + // Add the partition to the unassigned set to be re-assigned later on. + unassignedPartitions.add(new TopicIdPartition(topicId, partition)); + } + } + } + } else { + if (newAssignment == null) { + // If the new assignment is null, we create a deep copy of the + // original assignment so that we can alter it. + newAssignment = deepCopy(oldAssignment); + } + // Remove the entire topic. + newAssignment.remove(topicId); + } + } - /** - * Filters the current assignment of partitions for a given member based on certain criteria. - * - * Any partition that still belongs to the member's subscribed topics list is considered valid. - * - * @param currentMemberAssignment The map of topics to partitions currently assigned to the member. - * - * @return List of valid partitions after applying the filters. - */ - private List validCurrentMemberAssignment( - Map> currentMemberAssignment - ) { - List validCurrentAssignmentList = new ArrayList<>(); - currentMemberAssignment.forEach((topicId, partitions) -> { - if (subscribedTopicIds.contains(topicId)) { - partitions.forEach(partition -> { - TopicIdPartition topicIdPartition = new TopicIdPartition(topicId, partition); - validCurrentAssignmentList.add(topicIdPartition); - }); - } else { - LOG.debug("The topic " + topicId + " is no longer present in the subscribed topics list"); + if (quota > 0) { + unfilledMembers.add(new MemberWithRemainingQuota(memberId, quota)); } - }); - return validCurrentAssignmentList; + if (newAssignment == null) { + targetAssignment.put(memberId, new MemberAssignment(oldAssignment)); + } else { + targetAssignment.put(memberId, new MemberAssignment(newAssignment)); + } + } } /** - * Allocates the unassigned partitions to unfilled members in a round-robin fashion. + * Assign the unassigned partitions to the unfilled members. */ - private void unassignedPartitionsRoundRobinAssignment() { - Queue roundRobinMembers = new LinkedList<>(potentiallyUnfilledMembers.keySet()); - - // Partitions are sorted to ensure an even topic wise distribution across members. - // This not only balances the load but also makes partition-to-member mapping more predictable. - List sortedPartitionsList = unassignedPartitions.stream() - .sorted(Comparator.comparing(TopicIdPartition::topicId).thenComparing(TopicIdPartition::partitionId)) - .collect(Collectors.toList()); - - for (TopicIdPartition topicIdPartition : sortedPartitionsList) { - boolean assigned = false; - - for (int i = 0; i < roundRobinMembers.size() && !assigned; i++) { - String memberId = roundRobinMembers.poll(); - if (potentiallyUnfilledMembers.containsKey(memberId)) { - assigned = maybeAssignPartitionToMember(memberId, topicIdPartition); - } - // Only re-add the member to the end of the queue if it's still available for assignment. - if (potentiallyUnfilledMembers.containsKey(memberId)) { - roundRobinMembers.add(memberId); - } + private void assignRemainingPartitions() { + int unassignedPartitionIndex = 0; + + for (MemberWithRemainingQuota unfilledMember : unfilledMembers) { + String memberId = unfilledMember.memberId; + int remainingQuota = unfilledMember.remainingQuota; + + Map> newAssignment = targetAssignment.get(memberId).targetPartitions(); + if (isImmutableMap(newAssignment)) { + // If the new assignment is immutable, we must create a deep copy of it + // before altering it. + newAssignment = deepCopy(newAssignment); + targetAssignment.put(memberId, new MemberAssignment(newAssignment)); } - if (assigned) { - unassignedPartitions.remove(topicIdPartition); + for (int i = 0; i < remainingQuota && unassignedPartitionIndex < unassignedPartitions.size(); i++) { + TopicIdPartition unassignedTopicIdPartition = unassignedPartitions.get(unassignedPartitionIndex); + unassignedPartitionIndex++; + newAssignment + .computeIfAbsent(unassignedTopicIdPartition.topicId(), __ -> new HashSet<>()) + .add(unassignedTopicIdPartition.partitionId()); } } - } - /** - * Assigns the specified partition to the given member and updates the potentially unfilled members map. - * Only assign extra partitions once the member has met its minimum quota = total partitions / total members. - * - *
        - *
      1. If the minimum quota hasn't been met aka remaining > 0 directly assign the partition. - * After assigning the partition, if the min quota has been met aka remaining = 0, remove the member - * if there's no members left to receive an extra partition. Otherwise, keep it in the - * potentially unfilled map.
      2. - *
      3. If the minimum quota has been met and if there is potential to receive an extra partition, assign it. - * Remove the member from the potentially unfilled map since it has already received the extra partition - * and met the min quota.
      4. - *
      5. Else, don't assign the partition.
      6. - *
      - * - * @param memberId The Id of the member to which the partition will be assigned. - * @param topicIdPartition The topicIdPartition to be assigned. - * @return true if the assignment was successful, false otherwise. - */ - private boolean maybeAssignPartitionToMember(String memberId, TopicIdPartition topicIdPartition) { - int remaining = potentiallyUnfilledMembers.get(memberId); - boolean shouldAssign = false; - - // If the member hasn't met the minimum quota, set the flag for assignment. - // If member has met minimum quota and there's an extra partition available, set the flag for assignment. - if (remaining > 0) { - potentiallyUnfilledMembers.put(memberId, --remaining); - shouldAssign = true; - - // If the member meets the minimum quota due to this assignment, - // check if any extra partitions are available. - // Removing the member from the list reduces an iteration for when remaining = 0 but there's no extras left. - if (remaining == 0 && remainingMembersToGetAnExtraPartition == 0) { - potentiallyUnfilledMembers.remove(memberId); - } - } else if (remaining == 0 && remainingMembersToGetAnExtraPartition > 0) { - remainingMembersToGetAnExtraPartition--; - // Each member can only receive one extra partition, once they meet the minimum quota and receive an extra - // partition they can be removed from the potentially unfilled members map. - potentiallyUnfilledMembers.remove(memberId); - shouldAssign = true; + if (unassignedPartitionIndex < unassignedPartitions.size()) { + throw new PartitionAssignorException("Partitions were left unassigned"); } + } - // Assign the partition if flag is set. - if (shouldAssign) { - addPartitionToAssignment( - targetAssignment, - memberId, - topicIdPartition.topicId(), - topicIdPartition.partitionId() - ); - return true; + private static Map> deepCopy(Map> map) { + Map> copy = new HashMap<>(map.size()); + for (Map.Entry> entry : map.entrySet()) { + copy.put(entry.getKey(), new HashSet<>(entry.getValue())); } + return copy; + } + + private static class MemberWithRemainingQuota { + final String memberId; + final int remainingQuota; - // No assignment possible because the member met the minimum quota but - // number of members to receive an extra partition is zero. - return false; + MemberWithRemainingQuota( + String memberId, + int remainingQuota + ) { + this.memberId = memberId; + this.remainingQuota = remainingQuota; + } } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/PartitionAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/PartitionAssignor.java index 27b38cc6c4be4..f8b74bc218c17 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/PartitionAssignor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/PartitionAssignor.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.annotation.InterfaceStability; /** - * Server side partition assignor used by the GroupCoordinator. + * Server-side partition assignor used by the GroupCoordinator. * * The interface is kept in an internal module until KIP-848 is fully * implemented and ready to be released. @@ -34,12 +34,12 @@ public interface PartitionAssignor { /** * Assigns partitions to group members based on the given assignment specification and topic metadata. * - * @param assignmentSpec The assignment spec which includes member metadata. + * @param groupSpec The assignment spec which includes member metadata. * @param subscribedTopicDescriber The topic and partition metadata describer. * @return The new assignment for the group. */ GroupAssignment assign( - AssignmentSpec assignmentSpec, + GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber ) throws PartitionAssignorException; } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/PartitionAssignorException.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/PartitionAssignorException.java index 00fcd7d37e77a..482ad02f13aa5 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/PartitionAssignorException.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/PartitionAssignorException.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.errors.ApiException; /** - * Exception thrown by {@link PartitionAssignor#assign(AssignmentSpec)}. The exception + * Exception thrown by {@link PartitionAssignor#assign(GroupSpec, SubscribedTopicDescriber)}}. The exception * is only used internally. */ public class PartitionAssignorException extends ApiException { diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java index a1631f507a8ee..fe067901cbd67 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java @@ -49,7 +49,7 @@ * movements during reassignment. (Sticky) * */ -public class RangeAssignor implements PartitionAssignor { +public class RangeAssignor implements ConsumerGroupPartitionAssignor { public static final String RANGE_ASSIGNOR_NAME = "range"; @Override @@ -81,20 +81,20 @@ public MemberWithRemainingAssignments(String memberId, int remaining) { * Returns a map of topic Ids to a list of members subscribed to them, * based on the given assignment specification and metadata. * - * @param assignmentSpec The specification for member assignments. + * @param groupSpec The specification for member assignments. * @param subscribedTopicDescriber The metadata describer for subscribed topics and clusters. * @return A map of topic Ids to a list of member Ids subscribed to them. * * @throws PartitionAssignorException If a member is subscribed to a non-existent topic. */ private Map> membersPerTopic( - final AssignmentSpec assignmentSpec, + final GroupSpec groupSpec, final SubscribedTopicDescriber subscribedTopicDescriber ) { Map> membersPerTopic = new HashMap<>(); - Map membersData = assignmentSpec.members(); + Map membersData = groupSpec.members(); - if (assignmentSpec.subscriptionType().equals(HOMOGENEOUS)) { + if (groupSpec.subscriptionType().equals(HOMOGENEOUS)) { Set allMembers = membersData.keySet(); Collection topics = membersData.values().iterator().next().subscribedTopicIds(); @@ -139,7 +139,7 @@ private Map> membersPerTopic( */ @Override public GroupAssignment assign( - final AssignmentSpec assignmentSpec, + final GroupSpec groupSpec, final SubscribedTopicDescriber subscribedTopicDescriber ) throws PartitionAssignorException { @@ -147,7 +147,7 @@ public GroupAssignment assign( // Step 1 Map> membersPerTopic = membersPerTopic( - assignmentSpec, + groupSpec, subscribedTopicDescriber ); @@ -162,7 +162,7 @@ public GroupAssignment assign( List potentiallyUnfilledMembers = new ArrayList<>(); for (String memberId : membersForTopic) { - Set assignedPartitionsForTopic = assignmentSpec.members().get(memberId) + Set assignedPartitionsForTopic = groupSpec.members().get(memberId) .assignedPartitions().getOrDefault(topicId, Collections.emptySet()); int currentAssignmentSize = assignedPartitionsForTopic.size(); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java index aff9365dc70af..648b0161d648c 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java @@ -44,7 +44,7 @@ * @see OptimizedUniformAssignmentBuilder * @see GeneralUniformAssignmentBuilder */ -public class UniformAssignor implements PartitionAssignor { +public class UniformAssignor implements ConsumerGroupPartitionAssignor { private static final Logger LOG = LoggerFactory.getLogger(UniformAssignor.class); public static final String UNIFORM_ASSIGNOR_NAME = "uniform"; @@ -57,30 +57,28 @@ public String name() { * Perform the group assignment given the current members and * topics metadata. * - * @param assignmentSpec The assignment specification that included member metadata. + * @param groupSpec The assignment specification that included member metadata. * @param subscribedTopicDescriber The topic and cluster metadata describer {@link SubscribedTopicDescriber}. * @return The new target assignment for the group. */ @Override public GroupAssignment assign( - AssignmentSpec assignmentSpec, + GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber ) throws PartitionAssignorException { - AbstractUniformAssignmentBuilder assignmentBuilder; - - if (assignmentSpec.members().isEmpty()) + if (groupSpec.members().isEmpty()) return new GroupAssignment(Collections.emptyMap()); - if (assignmentSpec.subscriptionType().equals(HOMOGENEOUS)) { + if (groupSpec.subscriptionType().equals(HOMOGENEOUS)) { LOG.debug("Detected that all members are subscribed to the same set of topics, invoking the " + "optimized assignment algorithm"); - assignmentBuilder = new OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber); + return new OptimizedUniformAssignmentBuilder(groupSpec, subscribedTopicDescriber) + .build(); } else { LOG.debug("Detected that the members are subscribed to different sets of topics, invoking the " + "general assignment algorithm"); - assignmentBuilder = new GeneralUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber); + return new GeneralUniformAssignmentBuilder(groupSpec, subscribedTopicDescriber) + .buildAssignment(); } - - return assignmentBuilder.buildAssignment(); } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java index 9e334bb413b77..bbc544289b26b 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java @@ -170,6 +170,12 @@ public static class DeadlineAndEpoch { */ private final TimelineHashMap targetAssignment; + /** + * Reverse lookup map representing topic partitions with + * their current member assignments. + */ + private final TimelineHashMap> invertedTargetAssignment; + /** * The current partition epoch maps each topic-partitions to their current epoch where * the epoch is the epoch of their owners. When a member revokes a partition, it removes @@ -221,6 +227,7 @@ public ConsumerGroup( this.subscriptionType = new TimelineObject<>(snapshotRegistry, HOMOGENEOUS); this.targetAssignmentEpoch = new TimelineInteger(snapshotRegistry); this.targetAssignment = new TimelineHashMap<>(snapshotRegistry, 0); + this.invertedTargetAssignment = new TimelineHashMap<>(snapshotRegistry, 0); this.currentPartitionEpoch = new TimelineHashMap<>(snapshotRegistry, 0); this.metrics = Objects.requireNonNull(metrics); this.numClassicProtocolMembers = new TimelineInteger(snapshotRegistry); @@ -517,21 +524,89 @@ public Assignment targetAssignment(String memberId) { } /** - * Updates target assignment of a member. + * @return An immutable map containing all the topic partitions + * with their current member assignments. + */ + public Map> invertedTargetAssignment() { + return Collections.unmodifiableMap(invertedTargetAssignment); + } + + /** + * Updates the target assignment of a member. * * @param memberId The member id. * @param newTargetAssignment The new target assignment. */ public void updateTargetAssignment(String memberId, Assignment newTargetAssignment) { + updateInvertedTargetAssignment( + memberId, + targetAssignment.getOrDefault(memberId, new Assignment(Collections.emptyMap())), + newTargetAssignment + ); targetAssignment.put(memberId, newTargetAssignment); } + /** + * Updates the reverse lookup map of the target assignment. + * + * @param memberId The member Id. + * @param oldTargetAssignment The old target assignment. + * @param newTargetAssignment The new target assignment. + */ + private void updateInvertedTargetAssignment( + String memberId, + Assignment oldTargetAssignment, + Assignment newTargetAssignment + ) { + // Combine keys from both old and new assignments. + Set allTopicIds = new HashSet<>(); + allTopicIds.addAll(oldTargetAssignment.partitions().keySet()); + allTopicIds.addAll(newTargetAssignment.partitions().keySet()); + + for (Uuid topicId : allTopicIds) { + Set oldPartitions = oldTargetAssignment.partitions().getOrDefault(topicId, Collections.emptySet()); + Set newPartitions = newTargetAssignment.partitions().getOrDefault(topicId, Collections.emptySet()); + + TimelineHashMap topicPartitionAssignment = invertedTargetAssignment.computeIfAbsent( + topicId, k -> new TimelineHashMap<>(snapshotRegistry, Math.max(oldPartitions.size(), newPartitions.size())) + ); + + // Remove partitions that aren't present in the new assignment only if the partition is currently + // still assigned to the member in question. + // If p0 was moved from A to B, and the target assignment map was updated for B first, we don't want to + // remove the key p0 from the inverted map and undo the action when A eventually tries to update its assignment. + for (Integer partition : oldPartitions) { + if (!newPartitions.contains(partition) && memberId.equals(topicPartitionAssignment.get(partition))) { + topicPartitionAssignment.remove(partition); + } + } + + // Add partitions that are in the new assignment but not in the old assignment. + for (Integer partition : newPartitions) { + if (!oldPartitions.contains(partition)) { + topicPartitionAssignment.put(partition, memberId); + } + } + + if (topicPartitionAssignment.isEmpty()) { + invertedTargetAssignment.remove(topicId); + } else { + invertedTargetAssignment.put(topicId, topicPartitionAssignment); + } + } + } + /** * Removes the target assignment of a member. * * @param memberId The member id. */ public void removeTargetAssignment(String memberId) { + updateInvertedTargetAssignment( + memberId, + targetAssignment.getOrDefault(memberId, Assignment.EMPTY), + Assignment.EMPTY + ); targetAssignment.remove(memberId); } @@ -1001,6 +1076,29 @@ public Map computeSubscribedTopicNames( return subscribedTopicNames; } + /** + * Updates the subscription count with a set of members removed. + * + * @param removedMembers The set of removed members. + * + * @return Copy of the map of topics to the count of number of subscribers. + */ + public Map computeSubscribedTopicNames( + Set removedMembers + ) { + Map subscribedTopicNames = new HashMap<>(this.subscribedTopicNames); + if (removedMembers != null) { + removedMembers.forEach(removedMember -> + maybeUpdateSubscribedTopicNames( + subscribedTopicNames, + removedMember, + null + ) + ); + } + return subscribedTopicNames; + } + /** * Compute the subscription type of the consumer group. * @@ -1300,4 +1398,27 @@ public boolean allMembersUseClassicProtocolExcept(String memberId) { return numClassicProtocolMembers() == members().size() - 1 && !getOrMaybeCreateMember(memberId, false).useClassicProtocol(); } + + /** + * Checks whether the member has any unreleased partition. + * + * @param member The member to check. + * @return A boolean indicating whether the member has partitions in the target + * assignment that hasn't been revoked by other members. + */ + public boolean waitingOnUnreleasedPartition(ConsumerGroupMember member) { + if (member.state() == MemberState.UNRELEASED_PARTITIONS) { + for (Map.Entry> entry : targetAssignment().get(member.memberId()).partitions().entrySet()) { + Uuid topicId = entry.getKey(); + Set assignedPartitions = member.assignedPartitions().getOrDefault(topicId, Collections.emptySet()); + + for (int partition : entry.getValue()) { + if (!assignedPartitions.contains(partition) && currentPartitionEpoch(topicId, partition) != -1) { + return true; + } + } + } + } + return false; + } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java index 5f4ed7efc60fe..9846fe602680c 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java @@ -26,7 +26,6 @@ import java.util.ArrayList; import java.util.Collections; -import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -60,7 +59,7 @@ public static class Builder { private int rebalanceTimeoutMs = -1; private String clientId = ""; private String clientHost = ""; - private List subscribedTopicNames = Collections.emptyList(); + private Set subscribedTopicNames = Collections.emptySet(); private String subscribedTopicRegex = ""; private String serverAssignorName = null; private Map> assignedPartitions = Collections.emptyMap(); @@ -148,15 +147,13 @@ public Builder setClientHost(String clientHost) { return this; } - public Builder setSubscribedTopicNames(List subscribedTopicNames) { - this.subscribedTopicNames = subscribedTopicNames; - this.subscribedTopicNames.sort(Comparator.naturalOrder()); + public Builder setSubscribedTopicNames(List subscribedTopicList) { + if (subscribedTopicNames != null) this.subscribedTopicNames = new HashSet<>(subscribedTopicList); return this; } - public Builder maybeUpdateSubscribedTopicNames(Optional> subscribedTopicNames) { - this.subscribedTopicNames = subscribedTopicNames.orElse(this.subscribedTopicNames); - this.subscribedTopicNames.sort(Comparator.naturalOrder()); + public Builder maybeUpdateSubscribedTopicNames(Optional> subscribedTopicList) { + subscribedTopicList.ifPresent(list -> this.subscribedTopicNames = new HashSet<>(list)); return this; } @@ -299,7 +296,7 @@ public ConsumerGroupMember build() { /** * The list of subscriptions (topic names) configured by the member. */ - private final List subscribedTopicNames; + private final Set subscribedTopicNames; /** * The subscription pattern configured by the member. @@ -335,7 +332,7 @@ private ConsumerGroupMember( int rebalanceTimeoutMs, String clientId, String clientHost, - List subscribedTopicNames, + Set subscribedTopicNames, String subscribedTopicRegex, String serverAssignorName, MemberState state, @@ -419,7 +416,7 @@ public String clientHost() { /** * @return The list of subscribed topic names. */ - public List subscribedTopicNames() { + public Set subscribedTopicNames() { return subscribedTopicNames; } @@ -533,7 +530,7 @@ public ConsumerGroupDescribeResponseData.Member asConsumerGroupDescribeMember( .setClientId(clientId) .setInstanceId(instanceId) .setRackId(rackId) - .setSubscribedTopicNames(subscribedTopicNames) + .setSubscribedTopicNames(subscribedTopicNames == null ? null : new ArrayList<>(subscribedTopicNames)) .setSubscribedTopicRegex(subscribedTopicRegex); } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilder.java index 6cfe03b751629..57d6039fa0ba8 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilder.java @@ -19,22 +19,21 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.coordinator.group.CoordinatorRecord; import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec; -import org.apache.kafka.coordinator.group.assignor.AssignmentSpec; +import org.apache.kafka.coordinator.group.assignor.GroupSpecImpl; import org.apache.kafka.coordinator.group.assignor.SubscriptionType; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; import org.apache.kafka.coordinator.group.assignor.MemberAssignment; import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; import org.apache.kafka.coordinator.group.assignor.PartitionAssignorException; +import org.apache.kafka.image.TopicsImage; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.Set; import static org.apache.kafka.coordinator.group.CoordinatorRecordHelpers.newTargetAssignmentEpochRecord; import static org.apache.kafka.coordinator.group.CoordinatorRecordHelpers.newTargetAssignmentRecord; @@ -127,6 +126,17 @@ public Map targetAssignment() { */ private Map targetAssignment = Collections.emptyMap(); + /** + * Reverse lookup map representing topic partitions with + * their current member assignments. + */ + private Map> invertedTargetAssignment = Collections.emptyMap(); + + /** + * The topics image. + */ + private TopicsImage topicsImage = TopicsImage.EMPTY; + /** * The members which have been updated or deleted. Deleted members * are signaled by a null value. @@ -220,6 +230,32 @@ public TargetAssignmentBuilder withTargetAssignment( return this; } + /** + * Adds the existing topic partition assignments. + * + * @param invertedTargetAssignment The reverse lookup map of the current target assignment. + * @return This object. + */ + public TargetAssignmentBuilder withInvertedTargetAssignment( + Map> invertedTargetAssignment + ) { + this.invertedTargetAssignment = invertedTargetAssignment; + return this; + } + + /** + * Adds the topics image. + * + * @param topicsImage The topics image. + * @return This object. + */ + public TargetAssignmentBuilder withTopicsImage( + TopicsImage topicsImage + ) { + this.topicsImage = topicsImage; + return this; + } + /** * Adds or updates a member. This is useful when the updated member is * not yet materialized in memory. @@ -263,7 +299,7 @@ public TargetAssignmentResult build() throws PartitionAssignorException { members.forEach((memberId, member) -> memberSpecs.put(memberId, createAssignmentMemberSpec( member, targetAssignment.getOrDefault(memberId, Assignment.EMPTY), - subscriptionMetadata + topicsImage ))); // Update the member spec if updated or deleted members. @@ -284,7 +320,7 @@ public TargetAssignmentResult build() throws PartitionAssignorException { memberSpecs.put(memberId, createAssignmentMemberSpec( updatedMemberOrNull, assignment, - subscriptionMetadata + topicsImage )); } }); @@ -300,7 +336,11 @@ public TargetAssignmentResult build() throws PartitionAssignorException { // Compute the assignment. GroupAssignment newGroupAssignment = assignor.assign( - new AssignmentSpec(Collections.unmodifiableMap(memberSpecs), subscriptionType), + new GroupSpecImpl( + Collections.unmodifiableMap(memberSpecs), + subscriptionType, + invertedTargetAssignment + ), new SubscribedTopicMetadata(topicMetadataMap) ); @@ -353,23 +393,15 @@ private Assignment newMemberAssignment( } } - public static AssignmentMemberSpec createAssignmentMemberSpec( + static AssignmentMemberSpec createAssignmentMemberSpec( ConsumerGroupMember member, Assignment targetAssignment, - Map subscriptionMetadata + TopicsImage topicsImage ) { - Set subscribedTopics = new HashSet<>(); - member.subscribedTopicNames().forEach(topicName -> { - TopicMetadata topicMetadata = subscriptionMetadata.get(topicName); - if (topicMetadata != null) { - subscribedTopics.add(topicMetadata.id()); - } - }); - return new AssignmentMemberSpec( Optional.ofNullable(member.instanceId()), Optional.ofNullable(member.rackId()), - subscribedTopics, + new TopicIds(member.subscribedTopicNames(), topicsImage), targetAssignment.partitions() ); } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TopicIds.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TopicIds.java new file mode 100644 index 0000000000000..8485c1f560887 --- /dev/null +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TopicIds.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.group.consumer; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.image.TopicImage; +import org.apache.kafka.image.TopicsImage; + +import java.util.Collection; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Objects; +import java.util.Set; + +/** + * TopicIds is initialized with topic names (String) but exposes a Set of topic ids (Uuid) to the + * user and performs the conversion lazily with TopicsImage. + */ +public class TopicIds implements Set { + private final Set topicNames; + private final TopicsImage image; + + public TopicIds( + Set topicNames, + TopicsImage image + ) { + this.topicNames = Objects.requireNonNull(topicNames); + this.image = Objects.requireNonNull(image); + } + + @Override + public int size() { + return topicNames.size(); + } + + @Override + public boolean isEmpty() { + return topicNames.isEmpty(); + } + + @Override + public boolean contains(Object o) { + if (o instanceof Uuid) { + Uuid topicId = (Uuid) o; + TopicImage topicImage = image.getTopic(topicId); + if (topicImage == null) return false; + return topicNames.contains(topicImage.name()); + } + return false; + } + + private static class TopicIdIterator implements Iterator { + final Iterator iterator; + final TopicsImage image; + private Uuid next = null; + + private TopicIdIterator( + Iterator iterator, + TopicsImage image + ) { + this.iterator = Objects.requireNonNull(iterator); + this.image = Objects.requireNonNull(image); + } + + @Override + public boolean hasNext() { + if (next != null) return true; + Uuid result = null; + do { + if (!iterator.hasNext()) { + return false; + } + String next = iterator.next(); + TopicImage topicImage = image.getTopic(next); + if (topicImage != null) { + result = topicImage.id(); + } + } while (result == null); + next = result; + return true; + } + + @Override + public Uuid next() { + if (!hasNext()) throw new NoSuchElementException(); + Uuid result = next; + next = null; + return result; + } + } + + @Override + public Iterator iterator() { + return new TopicIdIterator(topicNames.iterator(), image); + } + + @Override + public Object[] toArray() { + throw new UnsupportedOperationException(); + } + + @Override + public T[] toArray(T[] a) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean add(Uuid o) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean remove(Object o) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean addAll(Collection c) { + throw new UnsupportedOperationException(); + } + + @Override + public void clear() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean removeAll(Collection c) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean retainAll(Collection c) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean containsAll(Collection c) { + for (Object o : c) { + if (!contains(o)) return false; + } + return true; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TopicIds uuids = (TopicIds) o; + + if (!Objects.equals(topicNames, uuids.topicNames)) return false; + return Objects.equals(image, uuids.image); + } + + @Override + public int hashCode() { + int result = topicNames.hashCode(); + result = 31 * result + image.hashCode(); + return result; + } +} diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java index ed6f649d61fcd..4207c94770b96 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java @@ -402,6 +402,19 @@ public void run() { timer.add(task); } + @Override + public void scheduleIfAbsent( + String key, + long delay, + TimeUnit unit, + boolean retry, + TimeoutOperation operation + ) { + if (!tasks.containsKey(key)) { + schedule(key, delay, unit, retry, 500, operation); + } + } + @Override public void cancel(String key) { TimerTask prevTask = tasks.remove(key); @@ -787,10 +800,10 @@ public void run() { byte magic = logConfig.recordVersion().value; int maxBatchSize = logConfig.maxMessageSize(); long currentTimeMs = time.milliseconds(); - ByteBuffer buffer = context.bufferSupplier.get(Math.min(16384, maxBatchSize)); + ByteBuffer buffer = context.bufferSupplier.get(Math.min(MIN_BUFFER_SIZE, maxBatchSize)); try { - MemoryRecordsBuilder builder = MemoryRecords.builder( + MemoryRecordsBuilder builder = new MemoryRecordsBuilder( buffer, magic, compression, @@ -801,7 +814,9 @@ public void run() { producerEpoch, 0, producerId != RecordBatch.NO_PRODUCER_ID, - RecordBatch.NO_PARTITION_LEADER_EPOCH + false, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + maxBatchSize ); // Apply the records to the state machine and add them to the batch. @@ -832,7 +847,8 @@ public void run() { ); } else { throw new RecordTooLargeException("Message batch size is " + builder.estimatedSizeInBytes() + - " bytes in append to partition $tp which exceeds the maximum configured size of $maxBatchSize."); + " bytes in append to partition " + tp + " which exceeds the maximum " + + "configured size of " + maxBatchSize + "."); } } @@ -1352,6 +1368,11 @@ public void onHighWatermarkUpdated( } } + /** + * 16KB. Used for initial buffer size for write operations. + */ + static final int MIN_BUFFER_SIZE = 16384; + /** * The log prefix. */ diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorTimer.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorTimer.java index 4c5b3aa8fd68c..ef5dfc6dc39ad 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorTimer.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorTimer.java @@ -66,6 +66,18 @@ interface TimeoutOperation { */ void schedule(String key, long delay, TimeUnit unit, boolean retry, long retryBackoff, TimeoutOperation operation); + /** + * Add an operation to the timer if there's no operation with the same key. + * + * @param key The key to identify this operation. + * @param delay The delay to wait before expiring. + * @param unit The delay unit. + * @param retry A boolean indicating whether the operation should + * be retried on failure. + * @param operation The operation to perform upon expiration. + */ + void scheduleIfAbsent(String key, long delay, TimeUnit unit, boolean retry, TimeoutOperation operation); + /** * Remove an operation corresponding to a given key. * diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java index 20337a4677581..31fa52ea7d158 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessor.java @@ -112,9 +112,6 @@ public MultiThreadedEventProcessor( */ private class EventProcessorThread extends Thread { private final Logger log; - private long pollStartMs; - private long timeSinceLastPollMs; - private long lastPollMs; EventProcessorThread( String name diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/AssignmentTestUtil.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/AssignmentTestUtil.java index 8b6a09b2ed84f..ffc5455ceafed 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/AssignmentTestUtil.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/AssignmentTestUtil.java @@ -17,16 +17,18 @@ package org.apache.kafka.coordinator.group; import org.apache.kafka.common.Uuid; +import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; import java.util.AbstractMap; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; -import java.util.TreeSet; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -41,13 +43,13 @@ public static Map.Entry> mkTopicAssignment( ); } - public static Map.Entry> mkSortedTopicAssignment( + public static Map.Entry> mkOrderedTopicAssignment( Uuid topicId, Integer... partitions ) { return new AbstractMap.SimpleEntry<>( topicId, - new TreeSet<>(Arrays.asList(partitions)) + new LinkedHashSet<>(Arrays.asList(partitions)) ); } @@ -55,18 +57,18 @@ public static Map.Entry> mkSortedTopicAssignment( public static Map> mkAssignment(Map.Entry>... entries) { Map> assignment = new HashMap<>(); for (Map.Entry> entry : entries) { - assignment.put(entry.getKey(), entry.getValue()); + assignment.put(entry.getKey(), Collections.unmodifiableSet(entry.getValue())); } - return assignment; + return Collections.unmodifiableMap(assignment); } @SafeVarargs - public static Map> mkSortedAssignment(Map.Entry>... entries) { + public static Map> mkOrderedAssignment(Map.Entry>... entries) { Map> assignment = new LinkedHashMap<>(); for (Map.Entry> entry : entries) { - assignment.put(entry.getKey(), entry.getValue()); + assignment.put(entry.getKey(), Collections.unmodifiableSet(entry.getValue())); } - return assignment; + return Collections.unmodifiableMap(assignment); } /** @@ -82,4 +84,33 @@ public static void assertAssignment( assertEquals(expectedAssignment.get(memberId), computedAssignmentForMember); }); } + + /** + * Generate a reverse look up map of partition to member target assignments from the given member spec. + * + * @param memberSpec A map where the key is the member Id and the value is an + * AssignmentMemberSpec object containing the member's partition assignments. + * @return Map of topic partition to member assignments. + */ + public static Map> invertedTargetAssignment( + Map memberSpec + ) { + Map> invertedTargetAssignment = new HashMap<>(); + for (Map.Entry memberEntry : memberSpec.entrySet()) { + String memberId = memberEntry.getKey(); + Map> topicsAndPartitions = memberEntry.getValue().assignedPartitions(); + + for (Map.Entry> topicEntry : topicsAndPartitions.entrySet()) { + Uuid topicId = topicEntry.getKey(); + Set partitions = topicEntry.getValue(); + + Map partitionMap = invertedTargetAssignment.computeIfAbsent(topicId, k -> new HashMap<>()); + + for (Integer partitionId : partitions) { + partitionMap.put(partitionId, memberId); + } + } + } + return invertedTargetAssignment; + } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/CoordinatorRecordHelpersTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/CoordinatorRecordHelpersTest.java index 77bad7a483499..900ba31839351 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/CoordinatorRecordHelpersTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/CoordinatorRecordHelpersTest.java @@ -69,8 +69,8 @@ import java.util.stream.Stream; import static org.apache.kafka.coordinator.group.Assertions.assertRecordEquals; -import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkSortedAssignment; -import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkSortedTopicAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkOrderedAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkOrderedTopicAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; import static org.apache.kafka.coordinator.group.CoordinatorRecordHelpers.newCurrentAssignmentRecord; import static org.apache.kafka.coordinator.group.CoordinatorRecordHelpers.newCurrentAssignmentTombstoneRecord; @@ -297,7 +297,7 @@ public void testNewTargetAssignmentRecord() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); - Map> partitions = mkSortedAssignment( + Map> partitions = mkOrderedAssignment( mkTopicAssignment(topicId1, 11, 12, 13), mkTopicAssignment(topicId2, 21, 22, 23) ); @@ -379,14 +379,14 @@ public void testNewCurrentAssignmentRecord() { Uuid topicId1 = Uuid.randomUuid(); Uuid topicId2 = Uuid.randomUuid(); - Map> assigned = mkSortedAssignment( - mkSortedTopicAssignment(topicId1, 11, 12, 13), - mkSortedTopicAssignment(topicId2, 21, 22, 23) + Map> assigned = mkOrderedAssignment( + mkOrderedTopicAssignment(topicId1, 11, 12, 13), + mkOrderedTopicAssignment(topicId2, 21, 22, 23) ); - Map> revoking = mkSortedAssignment( - mkSortedTopicAssignment(topicId1, 14, 15, 16), - mkSortedTopicAssignment(topicId2, 24, 25, 26) + Map> revoking = mkOrderedAssignment( + mkOrderedTopicAssignment(topicId1, 14, 15, 16), + mkOrderedTopicAssignment(topicId2, 24, 25, 26) ); CoordinatorRecord expectedRecord = new CoordinatorRecord( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java index da950c4d42618..e8906c4418a94 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.coordinator.group; import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; +import org.apache.kafka.coordinator.group.assignor.ConsumerGroupPartitionAssignor; import org.apache.kafka.coordinator.group.assignor.RangeAssignor; import org.junit.jupiter.api.Test; @@ -28,7 +28,7 @@ public class GroupCoordinatorConfigTest { @Test public void testConfigs() { - PartitionAssignor assignor = new RangeAssignor(); + ConsumerGroupPartitionAssignor assignor = new RangeAssignor(); GroupCoordinatorConfig config = new GroupCoordinatorConfig( 10, 30, diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 731fa5ca08a8f..6cd96458c647d 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -676,9 +676,10 @@ public void testHeartbeat() throws Exception { service.startup(() -> 1); - when(runtime.scheduleReadOperation( + when(runtime.scheduleWriteOperation( ArgumentMatchers.eq("classic-group-heartbeat"), ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)), + ArgumentMatchers.eq(Duration.ofMillis(5000)), ArgumentMatchers.any() )).thenReturn(CompletableFuture.completedFuture( new HeartbeatResponseData() @@ -708,9 +709,10 @@ public void testHeartbeatCoordinatorNotAvailableException() throws Exception { service.startup(() -> 1); - when(runtime.scheduleReadOperation( + when(runtime.scheduleWriteOperation( ArgumentMatchers.eq("classic-group-heartbeat"), ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)), + ArgumentMatchers.eq(Duration.ofMillis(5000)), ArgumentMatchers.any() )).thenReturn(FutureUtils.failedFuture( new CoordinatorLoadInProgressException(null) @@ -740,9 +742,10 @@ public void testHeartbeatCoordinatorException() throws Exception { service.startup(() -> 1); - when(runtime.scheduleReadOperation( + when(runtime.scheduleWriteOperation( ArgumentMatchers.eq("classic-group-heartbeat"), ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)), + ArgumentMatchers.eq(Duration.ofMillis(5000)), ArgumentMatchers.any() )).thenReturn(FutureUtils.failedFuture( new RebalanceInProgressException() diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 058cdf206db60..3664a7a61d295 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -61,9 +61,9 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.coordinator.group.MockCoordinatorTimer.ExpiredTimeout; import org.apache.kafka.coordinator.group.MockCoordinatorTimer.ScheduledTimeout; +import org.apache.kafka.coordinator.group.assignor.ConsumerGroupPartitionAssignor; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; import org.apache.kafka.coordinator.group.assignor.MemberAssignment; -import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; import org.apache.kafka.coordinator.group.assignor.PartitionAssignorException; import org.apache.kafka.coordinator.group.classic.ClassicGroupState; import org.apache.kafka.coordinator.group.consumer.Assignment; @@ -103,11 +103,11 @@ import static org.apache.kafka.common.protocol.Errors.NOT_COORDINATOR; import static org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest.LEAVE_GROUP_STATIC_MEMBER_EPOCH; -import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocol; import static org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection; import static org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH; import static org.apache.kafka.common.requests.JoinGroupRequest.UNKNOWN_MEMBER_ID; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.coordinator.group.Assertions.assertRecordEquals; import static org.apache.kafka.coordinator.group.Assertions.assertRecordsEquals; import static org.apache.kafka.coordinator.group.Assertions.assertResponseEquals; @@ -116,6 +116,7 @@ import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; import static org.apache.kafka.coordinator.group.GroupMetadataManager.appendGroupMetadataErrorToResponseError; import static org.apache.kafka.coordinator.group.GroupMetadataManager.classicGroupJoinKey; +import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupJoinKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupRebalanceTimeoutKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupSessionTimeoutKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.EMPTY_RESULT; @@ -149,7 +150,7 @@ public class GroupMetadataManagerTest { public void testConsumerHeartbeatRequestValidation() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .build(); Exception ex; @@ -239,7 +240,7 @@ public void testConsumerHeartbeatRequestValidation() { public void testMemberIdGeneration() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(MetadataImage.EMPTY) .build(); @@ -282,7 +283,7 @@ public void testUnknownGroupId() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .build(); assertThrows(GroupIdNotFoundException.class, () -> @@ -303,7 +304,7 @@ public void testUnknownMemberIdJoinsConsumerGroup() { String memberId = Uuid.randomUuid().toString(); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(new NoOpPartitionAssignor())) + .withConsumerGroupAssignors(Collections.singletonList(new NoOpPartitionAssignor())) .build(); // A first member joins to create the group. @@ -339,7 +340,7 @@ public void testConsumerGroupMemberEpochValidation() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .build(); ConsumerGroupMember member = new ConsumerGroupMember.Builder(memberId) @@ -425,7 +426,7 @@ public void testMemberJoinsEmptyConsumerGroup() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -515,7 +516,7 @@ public void testUpdatingSubscriptionTriggersNewTargetAssignment() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -617,7 +618,7 @@ public void testNewJoiningMemberTriggersNewTargetAssignment() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -757,7 +758,7 @@ public void testLeavingMemberBumpsGroupEpoch() { // Consumer group with two members. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -850,7 +851,7 @@ public void testGroupEpochBumpWhenNewStaticMemberJoins() { // Consumer group with two static members. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -1019,7 +1020,7 @@ public void testStaticMemberGetsBackAssignmentUponRejoin() { // Consumer group with two static members. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -1190,7 +1191,7 @@ public void testNoGroupEpochBumpWhenStaticMemberTemporarilyLeaves() { // Consumer group with two static members. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -1254,7 +1255,7 @@ public void testLeavingStaticMemberBumpsGroupEpoch() { // Consumer group with two static members. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -1347,7 +1348,7 @@ public void testShouldThrownUnreleasedInstanceIdExceptionWhenNewMemberJoinsWithI // Consumer group with one static member. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addRacks() @@ -1397,7 +1398,7 @@ public void testShouldThrownUnknownMemberIdExceptionWhenUnknownStaticMemberJoins // Consumer group with one static member. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .build()) @@ -1445,7 +1446,7 @@ public void testShouldThrowFencedInstanceIdExceptionWhenStaticMemberWithDifferen // Consumer group with one static member. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .build()) @@ -1487,7 +1488,7 @@ public void testConsumerGroupMemberEpochValidationForStaticMember() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .build(); ConsumerGroupMember member = new ConsumerGroupMember.Builder(memberId) @@ -1578,7 +1579,7 @@ public void testShouldThrowUnknownMemberIdExceptionWhenUnknownStaticMemberLeaves // Consumer group with one static member. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .build()) @@ -1624,7 +1625,7 @@ public void testShouldThrowFencedInstanceIdExceptionWhenStaticMemberWithDifferen // Consumer group with one static member. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .build()) @@ -1668,7 +1669,7 @@ public void testConsumerGroupHeartbeatFullResponse() { // Create a context with an empty consumer group. MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 2) .addRacks() @@ -1769,7 +1770,7 @@ public void testReconciliationProcess() { // Create a context with one consumer group containing two members. MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -2204,7 +2205,7 @@ public void testNewMemberIsRejectedWithMaximumMembersIsReached() { // Create a context with one consumer group containing two members. MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -2267,7 +2268,7 @@ public void testConsumerGroupStates() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)) .build(); @@ -2317,12 +2318,12 @@ public void testPartitionAssignorExceptionOnRegularHeartbeat() { Uuid barTopicId = Uuid.randomUuid(); String barTopicName = "bar"; - PartitionAssignor assignor = mock(PartitionAssignor.class); + ConsumerGroupPartitionAssignor assignor = mock(ConsumerGroupPartitionAssignor.class); when(assignor.name()).thenReturn("range"); when(assignor.assign(any(), any())).thenThrow(new PartitionAssignorException("Assignment failed.")); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -2356,7 +2357,7 @@ public void testSubscriptionMetadataRefreshedAfterGroupIsLoaded() { // Create a context with one consumer group containing one member. MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withConsumerGroupMetadataRefreshIntervalMs(5 * 60 * 1000) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) @@ -2467,7 +2468,7 @@ public void testSubscriptionMetadataRefreshedAgainAfterWriteFailure() { // Create a context with one consumer group containing one member. MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withConsumerGroupMetadataRefreshIntervalMs(5 * 60 * 1000) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) @@ -2591,7 +2592,7 @@ public void testGroupIdsByTopics() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .build(); assertEquals(Collections.emptySet(), context.groupMetadataManager.groupsSubscribedToTopic("foo")); @@ -2690,7 +2691,7 @@ public void testGroupIdsByTopics() { @Test public void testOnNewMetadataImageWithEmptyDelta() { GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .build(); MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY); @@ -2703,7 +2704,7 @@ public void testOnNewMetadataImageWithEmptyDelta() { @Test public void testOnNewMetadataImage() { GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .build(); // M1 in group 1 subscribes to a and b. @@ -2799,7 +2800,7 @@ public void testSessionTimeoutLifecycle() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addRacks() @@ -2874,7 +2875,7 @@ public void testSessionTimeoutExpiration() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addRacks() @@ -2938,7 +2939,7 @@ public void testSessionTimeoutExpirationStaticMember() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addRacks() @@ -3020,7 +3021,7 @@ public void testRebalanceTimeoutLifecycle() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 3) .addRacks() @@ -3175,7 +3176,7 @@ public void testRebalanceTimeoutExpiration() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 3) .addRacks() @@ -3312,7 +3313,7 @@ public void testOnLoaded() { String barTopicName = "bar"; GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -7452,7 +7453,7 @@ public void testStaticMemberHeartbeatLeaderWithInvalidMemberId() throws Exceptio .setMemberId(rebalanceResult.leaderId) .setGenerationId(rebalanceResult.generationId); - HeartbeatResponseData validHeartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + HeartbeatResponseData validHeartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.NONE.code(), validHeartbeatResponse.errorCode()); assertThrows(FencedInstanceIdException.class, () -> context.sendClassicGroupHeartbeat( @@ -7513,7 +7514,7 @@ public void testHeartbeatEmptyGroup() { .setMemberId("member-id") .setGenerationId(0); - HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), heartbeatResponse.errorCode()); } @@ -7560,7 +7561,7 @@ public void testHeartbeatDuringPreparingRebalance() throws Exception { .setGroupId("group-id") .setMemberId(memberId) .setGenerationId(0) - ); + ).response(); assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResponse.errorCode()); } @@ -7586,7 +7587,7 @@ public void testHeartbeatDuringCompletingRebalance() throws Exception { .setGroupId("group-id") .setMemberId(leaderJoinResponse.memberId()) .setGenerationId(leaderJoinResponse.generationId()) - ); + ).response(); assertEquals(new HeartbeatResponseData(), heartbeatResponse); } @@ -7616,7 +7617,7 @@ public void testValidHeartbeat() throws Exception { .setGroupId("group-id") .setMemberId(leaderJoinResponse.memberId()) .setGenerationId(leaderJoinResponse.generationId()) - ); + ).response(); assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); } @@ -7653,12 +7654,12 @@ public void testClassicGroupMemberHeartbeatMaintainsSession() throws Exception { .setMemberId(leaderJoinResponse.memberId()) .setGenerationId(leaderJoinResponse.generationId()); - HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(2500)); - heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); } @@ -7693,7 +7694,7 @@ public void testClassicGroupMemberSessionTimeoutDuringRebalance() throws Excepti .setMemberId(leaderJoinResponse.memberId()) .setGenerationId(leaderJoinResponse.generationId()); - HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResponse.errorCode()); // Advance clock by first member's session timeout. @@ -7771,7 +7772,7 @@ public void testRebalanceCompletesBeforeMemberJoins() throws Exception { for (int i = 0; i < 2; i++) { GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(2500)); - HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(firstMemberHeartbeatRequest); + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(firstMemberHeartbeatRequest).response(); assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResponse.errorCode()); } @@ -7813,7 +7814,7 @@ public void testRebalanceCompletesBeforeMemberJoins() throws Exception { firstMemberHeartbeatRequest .setMemberId(otherMemberId) .setGenerationId(2) - ); + ).response(); assertEquals(expectedError.code(), heartbeatResponse.errorCode()); } @@ -7855,7 +7856,7 @@ public void testRebalanceCompletesBeforeMemberJoins() throws Exception { firstMemberHeartbeatRequest .setMemberId(otherMemberId) .setGenerationId(3) - ); + ).response(); assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); } @@ -7872,7 +7873,7 @@ public void testSyncGroupEmptyAssignment() throws Exception { .setGroupId("group-id") .setMemberId(leaderJoinResponse.memberId()) .setGenerationId(leaderJoinResponse.generationId()) - ); + ).response(); assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); } @@ -7942,7 +7943,7 @@ public void testSecondMemberPartiallyJoinAndTimeout() throws Exception { for (int i = 0; i < 2; i++) { GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(2500)); - HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); } @@ -8203,7 +8204,7 @@ public void testHeartbeatDuringRebalanceCausesRebalanceInProgress() throws Excep .setMemberId(leaderJoinResponse.memberId()) .setGenerationId(leaderJoinResponse.generationId()); - HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest); + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest).response(); assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResponse.errorCode()); } @@ -8216,7 +8217,7 @@ public void testListGroups() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withConsumerGroup(new ConsumerGroupBuilder(consumerGroupId, 10)) .build(); @@ -8365,7 +8366,7 @@ public void testConsumerGroupDescribeNoErrors() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withConsumerGroup(new ConsumerGroupBuilder(consumerGroupIds.get(0), epoch)) .withConsumerGroup(new ConsumerGroupBuilder(consumerGroupIds.get(1), epoch) .withMember(memberBuilder.build())) @@ -8400,7 +8401,7 @@ public void testConsumerGroupDescribeWithErrors() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .build(); List actual = context.sendConsumerGroupDescribe(Collections.singletonList(groupId)); @@ -8428,7 +8429,7 @@ public void testConsumerGroupDescribeBeforeAndAfterCommittingOffset() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(metadataImage) .build(); @@ -9223,7 +9224,7 @@ public void testConsumerGroupRebalanceSensor() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -9353,7 +9354,7 @@ public void testConsumerGroupHeartbeatWithNonEmptyClassicGroup() { String classicGroupId = "classic-group-id"; String memberId = Uuid.randomUuid().toString(); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(new NoOpPartitionAssignor())) + .withConsumerGroupAssignors(Collections.singletonList(new NoOpPartitionAssignor())) .build(); ClassicGroup classicGroup = new ClassicGroup( new LogContext(), @@ -9382,7 +9383,7 @@ public void testConsumerGroupHeartbeatWithEmptyClassicGroup() { String classicGroupId = "classic-group-id"; String memberId = Uuid.randomUuid().toString(); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(new NoOpPartitionAssignor())) + .withConsumerGroupAssignors(Collections.singletonList(new NoOpPartitionAssignor())) .build(); ClassicGroup classicGroup = new ClassicGroup( new LogContext(), @@ -9493,7 +9494,7 @@ public void testConsumerGroupHeartbeatWithStableClassicGroup() { GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.UPGRADE) - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(metadataImage) .build(); @@ -9672,7 +9673,7 @@ public void testConsumerGroupHeartbeatWithPreparingRebalanceClassicGroup() throw GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.UPGRADE) - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(metadataImage) .build(); @@ -9919,7 +9920,7 @@ public void testConsumerGroupHeartbeatWithCompletingRebalanceClassicGroup() thro GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.UPGRADE) - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(metadataImage) .build(); @@ -10314,7 +10315,7 @@ public void testLastConsumerProtocolMemberLeavingConsumerGroup() { // Member 1 uses the classic protocol and member 2 uses the consumer protocol. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -10506,7 +10507,7 @@ public void testLastConsumerProtocolMemberSessionTimeoutInConsumerGroup() { // Member 1 uses the classic protocol and member 2 uses the consumer protocol. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -10696,7 +10697,7 @@ public void testLastConsumerProtocolMemberRebalanceTimeoutInConsumerGroup() { // Member 1 uses the classic protocol and member 2 uses the consumer protocol. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -10919,7 +10920,7 @@ public void testJoiningConsumerGroupWithNewDynamicMember() throws Exception { String memberId = Uuid.randomUuid().toString(); MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 2) .addTopic(barTopicId, barTopicName, 1) @@ -11069,7 +11070,7 @@ public void testJoiningConsumerGroupFailingToPersistRecords() throws Exception { )); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 2) .addRacks() @@ -11124,7 +11125,7 @@ public void testJoiningConsumerGroupWithNewStaticMember() throws Exception { String instanceId = "instance-id"; GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(new NoOpPartitionAssignor())) + .withConsumerGroupAssignors(Collections.singletonList(new NoOpPartitionAssignor())) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 2) .addTopic(barTopicId, barTopicName, 1) @@ -11220,7 +11221,7 @@ public void testJoiningConsumerGroupReplacingExistingStaticMember() throws Excep String memberId = Uuid.randomUuid().toString(); String instanceId = "instance-id"; GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(new NoOpPartitionAssignor())) + .withConsumerGroupAssignors(Collections.singletonList(new NoOpPartitionAssignor())) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 2) .addRacks() @@ -11326,7 +11327,7 @@ public void testJoiningConsumerGroupWithExistingStaticMemberAndNewSubscription() MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 2) .addTopic(barTopicId, barTopicName, 1) @@ -11556,7 +11557,7 @@ public void testReconciliationInJoiningConsumerGroupWithEagerProtocol() throws E MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 2) .addTopic(barTopicId, barTopicName, 1) @@ -11726,6 +11727,18 @@ public void testReconciliationInJoiningConsumerGroupWithEagerProtocol() throws E .setTopicPartitions(Collections.emptyList()) ); + // Member 1 heartbeats to be notified to rejoin. + assertEquals( + Errors.REBALANCE_IN_PROGRESS.code(), + context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setGenerationId(joinResponse1.generationId()) + ).response().errorCode() + ); + context.assertJoinTimeout(groupId, memberId1, 500); + // Member 1 rejoins to transition from UNRELEASED_PARTITIONS to STABLE. GroupMetadataManagerTestContext.JoinResult joinResult2 = context.sendClassicGroupJoin(request); ConsumerGroupMember expectedMember2 = new ConsumerGroupMember.Builder(expectedMember1) @@ -11743,6 +11756,7 @@ public void testReconciliationInJoiningConsumerGroupWithEagerProtocol() throws E assertEquals(expectedMember2.state(), group.getOrMaybeCreateMember(memberId1, false).state()); joinResult2.appendFuture.complete(null); + context.assertNoJoinTimeout(groupId, memberId1); JoinGroupResponseData joinResponse2 = joinResult2.joinFuture.get(); assertEquals( new JoinGroupResponseData() @@ -11785,7 +11799,7 @@ public void testReconciliationInJoiningConsumerGroupWithCooperativeProtocol() th MockPartitionAssignor assignor = new MockPartitionAssignor("range"); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroupAssignors(Collections.singletonList(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 2) .addTopic(barTopicId, barTopicName, 1) @@ -11943,6 +11957,18 @@ public void testReconciliationInJoiningConsumerGroupWithCooperativeProtocol() th Collections.singletonList(new TopicPartition(fooTopicName, 0)) ); + // Member 1 heartbeats to be notified to rejoin. + assertEquals( + Errors.REBALANCE_IN_PROGRESS.code(), + context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setGenerationId(joinResponse1.generationId()) + ).response().errorCode() + ); + context.assertJoinTimeout(groupId, memberId1, 500); + // Member 1 rejoins to transition from UNREVOKED_PARTITIONS to UNRELEASED_PARTITIONS. JoinGroupRequestData request2 = new GroupMetadataManagerTestContext.JoinGroupRequestBuilder() .withGroupId(groupId) @@ -11983,6 +12009,7 @@ public void testReconciliationInJoiningConsumerGroupWithCooperativeProtocol() th assertEquals(expectedMember2.state(), group.getOrMaybeCreateMember(memberId1, false).state()); joinResult2.appendFuture.complete(null); + context.assertNoJoinTimeout(groupId, memberId1); JoinGroupResponseData joinResponse2 = joinResult2.joinFuture.get(); assertEquals( new JoinGroupResponseData() @@ -12017,6 +12044,18 @@ public void testReconciliationInJoiningConsumerGroupWithCooperativeProtocol() th .setTopicPartitions(Collections.emptyList()) ); + // Member 1 heartbeats to be notified to rejoin. + assertEquals( + Errors.REBALANCE_IN_PROGRESS.code(), + context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setGenerationId(joinResponse2.generationId()) + ).response().errorCode() + ); + context.assertJoinTimeout(groupId, memberId1, 500); + // Member 1 rejoins to transition from UNRELEASED_PARTITIONS to STABLE. JoinGroupRequestData request3 = new GroupMetadataManagerTestContext.JoinGroupRequestBuilder() .withGroupId(groupId) @@ -12056,6 +12095,7 @@ public void testReconciliationInJoiningConsumerGroupWithCooperativeProtocol() th assertEquals(expectedMember3.state(), group.getOrMaybeCreateMember(memberId1, false).state()); joinResult3.appendFuture.complete(null); + context.assertNoJoinTimeout(groupId, memberId1); JoinGroupResponseData joinResponse3 = joinResult3.joinFuture.get(); assertEquals( new JoinGroupResponseData() @@ -12144,7 +12184,7 @@ public void testClassicGroupSyncToConsumerGroupWithAllConsumerProtocolVersions() // Member 1 uses the classic protocol and member 2 uses the consumer protocol. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) - .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -12182,7 +12222,7 @@ public void testClassicGroupSyncToConsumerGroupWithUnknownMemberId() throws Exce // Consumer group with a member that doesn't use the classic protocol. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) - .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId) .build())) @@ -12232,7 +12272,7 @@ public void testClassicGroupSyncToConsumerGroupWithFencedInstanceId() throws Exc // Consumer group with a static member. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) - .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId) .setInstanceId(instanceId) @@ -12261,7 +12301,7 @@ public void testClassicGroupSyncToConsumerGroupWithInconsistentGroupProtocol() t .setName("range") .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( new ConsumerPartitionAssignor.Subscription( - Arrays.asList("foo"), + Collections.singletonList("foo"), null, Collections.emptyList() ) @@ -12271,7 +12311,7 @@ public void testClassicGroupSyncToConsumerGroupWithInconsistentGroupProtocol() t // Consumer group with a member using the classic protocol. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) - .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId) .setClassicMemberMetadata( @@ -12326,7 +12366,7 @@ public void testClassicGroupSyncToConsumerGroupWithIllegalGeneration() throws Ex .setName("range") .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( new ConsumerPartitionAssignor.Subscription( - Arrays.asList("foo"), + Collections.singletonList("foo"), null, Collections.emptyList() ) @@ -12336,7 +12376,7 @@ public void testClassicGroupSyncToConsumerGroupWithIllegalGeneration() throws Ex // Consumer group with a member using the classic protocol. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) - .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) .withMember(new ConsumerGroupMember.Builder(memberId) .setClassicMemberMetadata( @@ -12369,7 +12409,7 @@ public void testClassicGroupSyncToConsumerGroupRebalanceInProgress() throws Exce .setName("range") .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( new ConsumerPartitionAssignor.Subscription( - Arrays.asList("foo"), + Collections.singletonList("foo"), null, Collections.emptyList() ) @@ -12380,9 +12420,10 @@ public void testClassicGroupSyncToConsumerGroupRebalanceInProgress() throws Exce // The group epoch is greater than the member epoch. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConsumerGroupMigrationPolicy(ConsumerGroupMigrationPolicy.DOWNGRADE) - .withAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) .withConsumerGroup(new ConsumerGroupBuilder(groupId, 11) .withMember(new ConsumerGroupMember.Builder(memberId) + .setRebalanceTimeoutMs(10000) .setClassicMemberMetadata( new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() .setSessionTimeoutMs(5000) @@ -12401,6 +12442,728 @@ public void testClassicGroupSyncToConsumerGroupRebalanceInProgress() throws Exce .withProtocolName("range") .build()) ); + context.assertJoinTimeout(groupId, memberId, 10000); + } + + @Test + public void testClassicGroupHeartbeatToConsumerGroupMaintainsSession() throws Exception { + String groupId = "group-id"; + String memberId = Uuid.randomUuid().toString(); + int sessionTimeout = 5000; + + List protocols = Collections.singletonList( + new ConsumerGroupMemberMetadataValue.ClassicProtocol() + .setName("range") + .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( + new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"), + null, + Collections.emptyList() + ) + ))) + ); + + // Consumer group with a member using the classic protocol. + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(sessionTimeout) + .setSupportedProtocols(protocols) + ) + .setMemberEpoch(10) + .build())) + .build(); + + // Heartbeat to schedule the session timeout. + HeartbeatRequestData request = new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setGenerationId(10); + context.sendClassicGroupHeartbeat(request); + context.assertSessionTimeout(groupId, memberId, sessionTimeout); + + // Advance clock by 1/2 of session timeout. + GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(sessionTimeout / 2)); + + HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(request).response(); + assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); + context.assertSessionTimeout(groupId, memberId, sessionTimeout); + + // Advance clock by 1/2 of session timeout. + GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(sessionTimeout / 2)); + + heartbeatResponse = context.sendClassicGroupHeartbeat(request).response(); + assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode()); + context.assertSessionTimeout(groupId, memberId, sessionTimeout); + } + + @Test + public void testClassicGroupHeartbeatToConsumerGroupRebalanceInProgress() throws Exception { + String groupId = "group-id"; + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + String memberId3 = Uuid.randomUuid().toString(); + Uuid fooTopicId = Uuid.randomUuid(); + Uuid barTopicId = Uuid.randomUuid(); + int sessionTimeout = 5000; + int rebalanceTimeout = 10000; + + List protocols = Collections.singletonList( + new ConsumerGroupMemberMetadataValue.ClassicProtocol() + .setName("range") + .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( + new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"), + null, + Collections.emptyList() + ) + ))) + ); + + // Member 1 has a member epoch smaller than the group epoch. + ConsumerGroupMember member1 = new ConsumerGroupMember.Builder(memberId1) + .setRebalanceTimeoutMs(rebalanceTimeout) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(sessionTimeout) + .setSupportedProtocols(protocols) + ) + .setMemberEpoch(9) + .build(); + + // Member 2 has unrevoked partition. + ConsumerGroupMember member2 = new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.UNREVOKED_PARTITIONS) + .setRebalanceTimeoutMs(rebalanceTimeout) + .setPartitionsPendingRevocation(mkAssignment(mkTopicAssignment(fooTopicId, 0))) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(sessionTimeout) + .setSupportedProtocols(protocols) + ) + .setMemberEpoch(10) + .build(); + + // Member 3 is in UNRELEASED_PARTITIONS and all the partitions in its target assignment are free. + ConsumerGroupMember member3 = new ConsumerGroupMember.Builder(memberId3) + .setState(MemberState.UNRELEASED_PARTITIONS) + .setRebalanceTimeoutMs(rebalanceTimeout) + .setAssignedPartitions(mkAssignment(mkTopicAssignment(barTopicId, 0))) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(sessionTimeout) + .setSupportedProtocols(protocols) + ) + .setMemberEpoch(10) + .build(); + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(member1) + .withMember(member2) + .withMember(member3) + .withAssignment(memberId3, mkAssignment(mkTopicAssignment(barTopicId, 0, 1, 2)))) + .build(); + + Arrays.asList(memberId1, memberId2, memberId3).forEach(memberId -> { + CoordinatorResult heartbeatResult = context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setGenerationId(memberId.equals(memberId1) ? 9 : 10) + ); + assertEquals(Collections.emptyList(), heartbeatResult.records()); + assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResult.response().errorCode()); + context.assertSessionTimeout(groupId, memberId, sessionTimeout); + context.assertJoinTimeout(groupId, memberId, rebalanceTimeout); + }); + } + + @Test + public void testClassicGroupHeartbeatToConsumerWithUnknownMember() { + String groupId = "group-id"; + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)) + .build(); + + assertThrows(UnknownMemberIdException.class, () -> context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId("unknown-member-id") + .setGenerationId(10) + )); + + assertThrows(UnknownMemberIdException.class, () -> context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId("unknown-member-id") + .setGroupInstanceId("unknown-instance-id") + .setGenerationId(10) + )); + } + + @Test + public void testClassicGroupHeartbeatToConsumerWithFencedInstanceId() { + String groupId = "group-id"; + String memberId = "member-id"; + String instanceId = "instance-id"; + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setInstanceId(instanceId) + .setMemberEpoch(10) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(5000) + .setSupportedProtocols(Collections.emptyList()) + ) + .build())) + .build(); + + assertThrows(FencedInstanceIdException.class, () -> context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId("unknown-member-id") + .setGroupInstanceId(instanceId) + .setGenerationId(10) + )); + } + + @Test + public void testClassicGroupHeartbeatToConsumerWithIllegalGenerationId() { + String groupId = "group-id"; + String memberId = "member-id"; + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setMemberEpoch(10) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(5000) + .setSupportedProtocols(Collections.emptyList()) + ) + .build())) + .build(); + + assertThrows(IllegalGenerationException.class, () -> context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setGenerationId(9) + )); + } + + @Test + public void testClassicGroupHeartbeatToConsumerWithMemberNotUsingClassicProtocol() { + String groupId = "group-id"; + String memberId = "member-id"; + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setMemberEpoch(10) + .build())) + .build(); + + assertThrows(UnknownMemberIdException.class, () -> context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setGenerationId(10) + )); + } + + @Test + public void testConsumerGroupMemberUsingClassicProtocolFencedWhenSessionTimeout() { + String groupId = "group-id"; + String memberId = Uuid.randomUuid().toString(); + int sessionTimeout = 5000; + + List protocols = Collections.singletonList( + new ConsumerGroupMemberMetadataValue.ClassicProtocol() + .setName("range") + .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( + new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"), + null, + Collections.emptyList() + ) + ))) + ); + + // Consumer group with a member using the classic protocol. + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(sessionTimeout) + .setSupportedProtocols(protocols) + ) + .setMemberEpoch(10) + .build())) + .build(); + + // Heartbeat to schedule the session timeout. + HeartbeatRequestData request = new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setGenerationId(10); + context.sendClassicGroupHeartbeat(request); + context.assertSessionTimeout(groupId, memberId, sessionTimeout); + + // Advance clock by session timeout + 1. + List> timeouts = context.sleep(sessionTimeout + 1); + + // The member is fenced from the group. + assertEquals(1, timeouts.size()); + MockCoordinatorTimer.ExpiredTimeout timeout = timeouts.get(0); + assertEquals(consumerGroupSessionTimeoutKey(groupId, memberId), timeout.key); + assertRecordsEquals( + Arrays.asList( + // The member is removed. + CoordinatorRecordHelpers.newCurrentAssignmentTombstoneRecord(groupId, memberId), + CoordinatorRecordHelpers.newTargetAssignmentTombstoneRecord(groupId, memberId), + CoordinatorRecordHelpers.newMemberSubscriptionTombstoneRecord(groupId, memberId), + + // The group epoch is bumped. + CoordinatorRecordHelpers.newGroupEpochRecord(groupId, 11) + ), + timeout.result.records() + ); + } + + @Test + public void testConsumerGroupMemberUsingClassicProtocolFencedWhenJoinTimeout() { + String groupId = "group-id"; + String memberId = Uuid.randomUuid().toString(); + int rebalanceTimeout = 500; + + List protocols = Collections.singletonList( + new ConsumerGroupMemberMetadataValue.ClassicProtocol() + .setName("range") + .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription( + new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"), + null, + Collections.emptyList() + ) + ))) + ); + + // Consumer group with a member using the classic protocol whose member epoch is smaller than the group epoch. + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setRebalanceTimeoutMs(rebalanceTimeout) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(5000) + .setSupportedProtocols(protocols) + ) + .setMemberEpoch(9) + .build())) + .build(); + + // Heartbeat to schedule the join timeout. + HeartbeatRequestData request = new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setGenerationId(9); + assertEquals( + Errors.REBALANCE_IN_PROGRESS.code(), + context.sendClassicGroupHeartbeat(request).response().errorCode() + ); + context.assertSessionTimeout(groupId, memberId, 5000); + context.assertJoinTimeout(groupId, memberId, rebalanceTimeout); + + // Advance clock by rebalance timeout + 1. + List> timeouts = context.sleep(rebalanceTimeout + 1); + + // The member is fenced from the group. + assertEquals(1, timeouts.size()); + MockCoordinatorTimer.ExpiredTimeout timeout = timeouts.get(0); + assertEquals(consumerGroupJoinKey(groupId, memberId), timeout.key); + assertRecordsEquals( + Arrays.asList( + // The member is removed. + CoordinatorRecordHelpers.newCurrentAssignmentTombstoneRecord(groupId, memberId), + CoordinatorRecordHelpers.newTargetAssignmentTombstoneRecord(groupId, memberId), + CoordinatorRecordHelpers.newMemberSubscriptionTombstoneRecord(groupId, memberId), + + // The group epoch is bumped. + CoordinatorRecordHelpers.newGroupEpochRecord(groupId, 11) + ), + timeout.result.records() + ); + } + + @Test + public void testConsumerGroupMemberUsingClassicProtocolBatchLeaveGroup() { + String groupId = "group-id"; + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + String memberId3 = Uuid.randomUuid().toString(); + String instanceId2 = "instance-id-2"; + String instanceId3 = "instance-id-3"; + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + Uuid barTopicId = Uuid.randomUuid(); + String barTopicName = "bar"; + + List protocol1 = Collections.singletonList( + new ConsumerGroupMemberMetadataValue.ClassicProtocol() + .setName("range") + .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Arrays.asList(fooTopicName, barTopicName), + null, + Collections.singletonList(new TopicPartition(fooTopicName, 0)) + )))) + ); + List protocol2 = Collections.singletonList( + new ConsumerGroupMemberMetadataValue.ClassicProtocol() + .setName("range") + .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Arrays.asList(fooTopicName, barTopicName), + null, + Collections.singletonList(new TopicPartition(fooTopicName, 1)) + )))) + ); + + ConsumerGroupMember member1 = new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setRebalanceTimeoutMs(45000) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(5000) + .setSupportedProtocols(protocol1) + ) + .setAssignedPartitions(mkAssignment(mkTopicAssignment(fooTopicId, 0))) + .build(); + ConsumerGroupMember member2 = new ConsumerGroupMember.Builder(memberId2) + .setInstanceId(instanceId2) + .setState(MemberState.STABLE) + .setMemberEpoch(9) + .setPreviousMemberEpoch(8) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setRebalanceTimeoutMs(45000) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(5000) + .setSupportedProtocols(protocol2) + ) + .setAssignedPartitions(mkAssignment(mkTopicAssignment(fooTopicId, 1))) + .build(); + ConsumerGroupMember member3 = new ConsumerGroupMember.Builder(memberId3) + .setInstanceId(instanceId3) + .setState(MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setRebalanceTimeoutMs(45000) + .setAssignedPartitions(mkAssignment(mkTopicAssignment(barTopicId, 0))) + .build(); + + // Consumer group with three members. + // Dynamic member 1 uses the classic protocol. + // Static member 2 uses the classic protocol. + // Static member 3 uses the consumer protocol. + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 2) + .addTopic(barTopicId, barTopicName, 1) + .addRacks() + .build()) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(member1) + .withMember(member2) + .withMember(member3) + .withAssignment(memberId1, mkAssignment(mkTopicAssignment(fooTopicId, 0))) + .withAssignment(memberId2, mkAssignment(mkTopicAssignment(fooTopicId, 1))) + .withAssignment(memberId3, mkAssignment(mkTopicAssignment(barTopicId, 0))) + .withAssignmentEpoch(10)) + .build(); + context.groupMetadataManager.consumerGroup(groupId).setMetadataRefreshDeadline(Long.MAX_VALUE, 10); + context.replay(CoordinatorRecordHelpers.newGroupSubscriptionMetadataRecord(groupId, new HashMap() { + { + put(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 2, mkMapOfPartitionRacks(2))); + put(barTopicName, new TopicMetadata(barTopicId, barTopicName, 1, mkMapOfPartitionRacks(1))); + } + })); + + // Member 1 joins to schedule the sync timeout and the heartbeat timeout. + context.sendClassicGroupJoin( + new GroupMetadataManagerTestContext.JoinGroupRequestBuilder() + .withGroupId(groupId) + .withMemberId(memberId1) + .withRebalanceTimeoutMs(member1.rebalanceTimeoutMs()) + .withSessionTimeoutMs(member1.classicMemberMetadata().get().sessionTimeoutMs()) + .withProtocols(GroupMetadataManagerTestContext.toConsumerProtocol( + Arrays.asList(fooTopicName, barTopicName), + Collections.singletonList(new TopicPartition(fooTopicName, 0)))) + .build() + ).appendFuture.complete(null); + context.assertSyncTimeout(groupId, memberId1, member1.rebalanceTimeoutMs()); + context.assertSessionTimeout(groupId, memberId1, member1.classicMemberMetadata().get().sessionTimeoutMs()); + + // Member 2 heartbeats to schedule the join timeout and the heartbeat timeout. + context.sendClassicGroupHeartbeat( + new HeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setGenerationId(9) + ); + context.assertJoinTimeout(groupId, memberId2, member2.rebalanceTimeoutMs()); + context.assertSessionTimeout(groupId, memberId2, member2.classicMemberMetadata().get().sessionTimeoutMs()); + + // Member 1 and member 2 leave the group. + CoordinatorResult leaveResult = context.sendClassicGroupLeave( + new LeaveGroupRequestData() + .setGroupId("group-id") + .setMembers(Arrays.asList( + // Valid member id. + new MemberIdentity() + .setMemberId(memberId1), + new MemberIdentity() + .setGroupInstanceId(instanceId2), + // Member that doesn't use the classic protocol. + new MemberIdentity() + .setMemberId(memberId3) + .setGroupInstanceId(instanceId3), + // Unknown member id. + new MemberIdentity() + .setMemberId("unknown-member-id"), + new MemberIdentity() + .setGroupInstanceId("unknown-instance-id"), + // Fenced instance id. + new MemberIdentity() + .setMemberId("unknown-member-id") + .setGroupInstanceId(instanceId3) + )) + ); + + assertEquals( + new LeaveGroupResponseData() + .setMembers(Arrays.asList( + new LeaveGroupResponseData.MemberResponse() + .setGroupInstanceId(null) + .setMemberId(memberId1), + new LeaveGroupResponseData.MemberResponse() + .setGroupInstanceId(instanceId2) + .setMemberId(memberId2), + new LeaveGroupResponseData.MemberResponse() + .setGroupInstanceId(instanceId3) + .setMemberId(memberId3) + .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()), + new LeaveGroupResponseData.MemberResponse() + .setGroupInstanceId(null) + .setMemberId("unknown-member-id") + .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()), + new LeaveGroupResponseData.MemberResponse() + .setGroupInstanceId("unknown-instance-id") + .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()), + new LeaveGroupResponseData.MemberResponse() + .setGroupInstanceId(instanceId3) + .setMemberId("unknown-member-id") + .setErrorCode(Errors.FENCED_INSTANCE_ID.code()) + )), + leaveResult.response() + ); + + List expectedRecords = Arrays.asList( + // Remove member 1 + CoordinatorRecordHelpers.newCurrentAssignmentTombstoneRecord(groupId, memberId1), + CoordinatorRecordHelpers.newTargetAssignmentTombstoneRecord(groupId, memberId1), + CoordinatorRecordHelpers.newMemberSubscriptionTombstoneRecord(groupId, memberId1), + // Remove member 2. + CoordinatorRecordHelpers.newCurrentAssignmentTombstoneRecord(groupId, memberId2), + CoordinatorRecordHelpers.newTargetAssignmentTombstoneRecord(groupId, memberId2), + CoordinatorRecordHelpers.newMemberSubscriptionTombstoneRecord(groupId, memberId2), + // Bump the group epoch. + CoordinatorRecordHelpers.newGroupEpochRecord(groupId, 11) + ); + assertEquals(expectedRecords, leaveResult.records()); + + context.assertNoSessionTimeout(groupId, memberId1); + context.assertNoSyncTimeout(groupId, memberId1); + context.assertNoSessionTimeout(groupId, memberId2); + context.assertNoJoinTimeout(groupId, memberId2); + } + + @Test + public void testConsumerGroupMemberUsingClassicProtocolBatchLeaveGroupUpdatingSubscriptionMetadata() { + String groupId = "group-id"; + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + Uuid barTopicId = Uuid.randomUuid(); + String barTopicName = "bar"; + + List protocol = Collections.singletonList( + new ConsumerGroupMemberMetadataValue.ClassicProtocol() + .setName("range") + .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Arrays.asList(fooTopicName, barTopicName), + null, + Collections.singletonList(new TopicPartition(fooTopicName, 0)) + )))) + ); + + ConsumerGroupMember member1 = new ConsumerGroupMember.Builder(memberId1) + .setState(MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setRebalanceTimeoutMs(45000) + .setClassicMemberMetadata( + new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() + .setSessionTimeoutMs(5000) + .setSupportedProtocols(protocol) + ) + .setAssignedPartitions(mkAssignment(mkTopicAssignment(fooTopicId, 0))) + .build(); + ConsumerGroupMember member2 = new ConsumerGroupMember.Builder(memberId2) + .setState(MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo")) + .setServerAssignorName("range") + .setRebalanceTimeoutMs(45000) + .setAssignedPartitions(mkAssignment(mkTopicAssignment(barTopicId, 0))) + .build(); + + // Consumer group with two members. + // Member 1 uses the classic protocol and member 2 uses the consumer protocol. + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 2) + .addTopic(barTopicId, barTopicName, 1) + .addRacks() + .build()) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(member1) + .withMember(member2) + .withAssignment(memberId1, mkAssignment(mkTopicAssignment(fooTopicId, 0))) + .withAssignment(memberId2, mkAssignment(mkTopicAssignment(barTopicId, 0))) + .withAssignmentEpoch(10)) + .build(); + context.groupMetadataManager.consumerGroup(groupId).setMetadataRefreshDeadline(Long.MAX_VALUE, 10); + context.replay(CoordinatorRecordHelpers.newGroupSubscriptionMetadataRecord(groupId, new HashMap() { + { + put(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 2, mkMapOfPartitionRacks(2))); + put(barTopicName, new TopicMetadata(barTopicId, barTopicName, 1, mkMapOfPartitionRacks(1))); + } + })); + + // Member 1 leaves the group. + CoordinatorResult leaveResult = context.sendClassicGroupLeave( + new LeaveGroupRequestData() + .setGroupId("group-id") + .setMembers(Collections.singletonList( + new MemberIdentity() + .setMemberId(memberId1) + )) + ); + + assertEquals( + new LeaveGroupResponseData() + .setMembers(Collections.singletonList( + new LeaveGroupResponseData.MemberResponse() + .setGroupInstanceId(null) + .setMemberId(memberId1) + )), + leaveResult.response() + ); + + List expectedRecords = Arrays.asList( + // Remove member 1 + CoordinatorRecordHelpers.newCurrentAssignmentTombstoneRecord(groupId, memberId1), + CoordinatorRecordHelpers.newTargetAssignmentTombstoneRecord(groupId, memberId1), + CoordinatorRecordHelpers.newMemberSubscriptionTombstoneRecord(groupId, memberId1), + // Update the subscription metadata. + CoordinatorRecordHelpers.newGroupSubscriptionMetadataRecord(groupId, new HashMap() { + { + put(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 2, mkMapOfPartitionRacks(2))); + } + }), + // Bump the group epoch. + CoordinatorRecordHelpers.newGroupEpochRecord(groupId, 11) + ); + assertEquals(expectedRecords, leaveResult.records()); + } + + @Test + public void testClassicGroupLeaveToConsumerGroupWithoutValidLeaveGroupMember() { + String groupId = "group-id"; + String memberId = Uuid.randomUuid().toString(); + + // Consumer group without member using the classic protocol. + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withConsumerGroupAssignors(Collections.singletonList(new MockPartitionAssignor("range"))) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .build())) + .build(); + + // Send leave request without valid member. + CoordinatorResult leaveResult = context.sendClassicGroupLeave( + new LeaveGroupRequestData() + .setGroupId("group-id") + .setMembers(Arrays.asList( + new MemberIdentity() + .setMemberId("unknown-member-id"), + new MemberIdentity() + .setMemberId(memberId) + )) + ); + + assertEquals( + new LeaveGroupResponseData() + .setMembers(Arrays.asList( + new LeaveGroupResponseData.MemberResponse() + .setGroupInstanceId(null) + .setMemberId("unknown-member-id") + .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()), + new LeaveGroupResponseData.MemberResponse() + .setGroupInstanceId(null) + .setMemberId(memberId) + .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()) + )), + leaveResult.response() + ); + + assertEquals(Collections.emptyList(), leaveResult.records()); } private static void checkJoinGroupResponse( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java index ce57498b21203..e6268edc34248 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java @@ -46,7 +46,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; +import org.apache.kafka.coordinator.group.assignor.ConsumerGroupPartitionAssignor; import org.apache.kafka.coordinator.group.classic.ClassicGroup; import org.apache.kafka.coordinator.group.consumer.ConsumerGroup; import org.apache.kafka.coordinator.group.consumer.ConsumerGroupBuilder; @@ -89,6 +89,7 @@ import static org.apache.kafka.coordinator.group.Assertions.assertSyncGroupResponseEquals; import static org.apache.kafka.coordinator.group.GroupMetadataManager.EMPTY_RESULT; import static org.apache.kafka.coordinator.group.GroupMetadataManager.classicGroupHeartbeatKey; +import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupJoinKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupRebalanceTimeoutKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupSessionTimeoutKey; import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupSyncKey; @@ -376,7 +377,7 @@ public static class Builder { final private LogContext logContext = new LogContext(); final private SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); private MetadataImage metadataImage; - private List consumerGroupAssignors = Collections.singletonList(new MockPartitionAssignor("range")); + private List consumerGroupAssignors = Collections.singletonList(new MockPartitionAssignor("range")); final private List consumerGroupBuilders = new ArrayList<>(); private int consumerGroupMaxSize = Integer.MAX_VALUE; private int consumerGroupMetadataRefreshIntervalMs = Integer.MAX_VALUE; @@ -393,7 +394,7 @@ public Builder withMetadataImage(MetadataImage metadataImage) { return this; } - public Builder withAssignors(List assignors) { + public Builder withConsumerGroupAssignors(List assignors) { this.consumerGroupAssignors = assignors; return this; } @@ -619,6 +620,27 @@ public void assertNoRebalanceTimeout( assertNull(timeout); } + public MockCoordinatorTimer.ScheduledTimeout assertJoinTimeout( + String groupId, + String memberId, + long delayMs + ) { + MockCoordinatorTimer.ScheduledTimeout timeout = + timer.timeout(consumerGroupJoinKey(groupId, memberId)); + assertNotNull(timeout); + assertEquals(time.milliseconds() + delayMs, timeout.deadlineMs); + return timeout; + } + + public void assertNoJoinTimeout( + String groupId, + String memberId + ) { + MockCoordinatorTimer.ScheduledTimeout timeout = + timer.timeout(consumerGroupJoinKey(groupId, memberId)); + assertNull(timeout); + } + public MockCoordinatorTimer.ScheduledTimeout assertSyncTimeout( String groupId, String memberId, @@ -1112,7 +1134,7 @@ public void verifySessionExpiration(ClassicGroup group, int timeoutMs) { assertEquals(0, group.size()); } - public HeartbeatResponseData sendClassicGroupHeartbeat( + public CoordinatorResult sendClassicGroupHeartbeat( HeartbeatRequestData request ) { RequestContext context = new RequestContext( @@ -1164,7 +1186,7 @@ public void verifyHeartbeat( if (expectedError == Errors.UNKNOWN_MEMBER_ID) { assertThrows(UnknownMemberIdException.class, () -> sendClassicGroupHeartbeat(request)); } else { - HeartbeatResponseData response = sendClassicGroupHeartbeat(request); + HeartbeatResponseData response = sendClassicGroupHeartbeat(request).response(); assertEquals(expectedError.code(), response.errorCode()); } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MockCoordinatorTimer.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MockCoordinatorTimer.java index 439da4bbf473f..b4942c06dc550 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MockCoordinatorTimer.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MockCoordinatorTimer.java @@ -130,6 +130,19 @@ public void schedule( schedule(key, delay, unit, retry, 500L, operation); } + @Override + public void scheduleIfAbsent( + String key, + long delay, + TimeUnit unit, + boolean retry, + TimeoutOperation operation + ) { + if (!timeoutMap.containsKey(key)) { + schedule(key, delay, unit, retry, 500L, operation); + } + } + /** * Cancels a timeout. */ diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MockPartitionAssignor.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MockPartitionAssignor.java index 736ee4395a58b..f084bb86e1ce5 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MockPartitionAssignor.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MockPartitionAssignor.java @@ -17,9 +17,9 @@ package org.apache.kafka.coordinator.group; import org.apache.kafka.common.Uuid; -import org.apache.kafka.coordinator.group.assignor.AssignmentSpec; +import org.apache.kafka.coordinator.group.assignor.ConsumerGroupPartitionAssignor; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; -import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; +import org.apache.kafka.coordinator.group.assignor.GroupSpec; import org.apache.kafka.coordinator.group.assignor.PartitionAssignorException; import org.apache.kafka.coordinator.group.assignor.SubscribedTopicDescriber; @@ -27,7 +27,7 @@ import java.util.Objects; import java.util.Set; -public class MockPartitionAssignor implements PartitionAssignor { +public class MockPartitionAssignor implements ConsumerGroupPartitionAssignor { private final String name; private GroupAssignment prepareGroupAssignment = null; @@ -45,7 +45,7 @@ public String name() { } @Override - public GroupAssignment assign(AssignmentSpec assignmentSpec, SubscribedTopicDescriber subscribedTopicDescriber) throws PartitionAssignorException { + public GroupAssignment assign(GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber) throws PartitionAssignorException { return prepareGroupAssignment; } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/NoOpPartitionAssignor.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/NoOpPartitionAssignor.java index 05b8cdc4c76c6..2cf8309a72a40 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/NoOpPartitionAssignor.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/NoOpPartitionAssignor.java @@ -16,16 +16,16 @@ */ package org.apache.kafka.coordinator.group; -import org.apache.kafka.coordinator.group.assignor.AssignmentSpec; +import org.apache.kafka.coordinator.group.assignor.ConsumerGroupPartitionAssignor; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; +import org.apache.kafka.coordinator.group.assignor.GroupSpec; import org.apache.kafka.coordinator.group.assignor.MemberAssignment; -import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; import org.apache.kafka.coordinator.group.assignor.SubscribedTopicDescriber; import java.util.Map; import java.util.stream.Collectors; -public class NoOpPartitionAssignor implements PartitionAssignor { +public class NoOpPartitionAssignor implements ConsumerGroupPartitionAssignor { static final String NAME = "no-op"; @Override @@ -34,8 +34,8 @@ public String name() { } @Override - public GroupAssignment assign(AssignmentSpec assignmentSpec, SubscribedTopicDescriber subscribedTopicDescriber) { - return new GroupAssignment(assignmentSpec.members().entrySet() + public GroupAssignment assign(GroupSpec groupSpec, SubscribedTopicDescriber subscribedTopicDescriber) { + return new GroupAssignment(groupSpec.members().entrySet() .stream() .collect(Collectors.toMap( Map.Entry::getKey, diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilderTest.java index 13f860209db1d..f5a7be24bf2a9 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilderTest.java @@ -21,7 +21,6 @@ import org.apache.kafka.coordinator.group.consumer.TopicMetadata; import org.junit.jupiter.api.Test; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -29,9 +28,11 @@ import java.util.Set; import java.util.TreeMap; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.assertAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.invertedTargetAssignment; import static org.apache.kafka.coordinator.group.CoordinatorRecordHelpersTest.mkMapOfPartitionRacks; import static org.apache.kafka.coordinator.group.assignor.SubscriptionType.HETEROGENEOUS; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -69,18 +70,26 @@ public void testTwoMembersNoTopicSubscription() { members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.emptyList(), + Collections.emptySet(), Collections.emptyMap() )); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.emptyList(), + Collections.emptySet(), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HETEROGENEOUS); - GroupAssignment groupAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HETEROGENEOUS, + Collections.emptyMap() + ); + + GroupAssignment groupAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); assertEquals(Collections.emptyMap(), groupAssignment.members()); } @@ -103,20 +112,24 @@ public void testTwoMembersSubscribedToNonexistentTopics() { members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic3Uuid), + Collections.singleton(topic3Uuid), Collections.emptyMap() )); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic2Uuid), + Collections.singleton(topic2Uuid), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HETEROGENEOUS); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HETEROGENEOUS, + Collections.emptyMap() + ); assertThrows(PartitionAssignorException.class, - () -> assignor.assign(assignmentSpec, subscribedTopicMetadata)); + () -> assignor.assign(groupSpec, subscribedTopicMetadata)); } @Test @@ -139,20 +152,27 @@ public void testFirstAssignmentTwoMembersTwoTopics() { members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic3Uuid), + mkSet(topic1Uuid, topic3Uuid), Collections.emptyMap() )); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic3Uuid), + Collections.singleton(topic3Uuid), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HETEROGENEOUS); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HETEROGENEOUS, + Collections.emptyMap() + ); SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( @@ -186,26 +206,33 @@ public void testFirstAssignmentNumMembersGreaterThanTotalNumPartitions() { members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic3Uuid), + Collections.singleton(topic3Uuid), Collections.emptyMap() )); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic3Uuid), + Collections.singleton(topic3Uuid), Collections.emptyMap() )); members.put(memberC, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic1Uuid), + Collections.singleton(topic1Uuid), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HETEROGENEOUS); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HETEROGENEOUS, + Collections.emptyMap() + ); SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); // Topic 3 has 2 partitions but three members subscribed to it - one of them should not get an assignment. Map>> expectedAssignment = new HashMap<>(); @@ -254,7 +281,7 @@ public void testReassignmentForTwoMembersThreeTopicsGivenUnbalancedPrevAssignmen members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.of("rack0"), - Collections.singletonList(topic1Uuid), + Collections.singleton(topic1Uuid), currentAssignmentForA )); @@ -267,7 +294,7 @@ public void testReassignmentForTwoMembersThreeTopicsGivenUnbalancedPrevAssignmen members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.of("rack1"), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), currentAssignmentForB )); @@ -281,14 +308,21 @@ public void testReassignmentForTwoMembersThreeTopicsGivenUnbalancedPrevAssignmen members.put(memberC, new AssignmentMemberSpec( Optional.empty(), Optional.of("rack2"), - Arrays.asList(topic1Uuid, topic2Uuid, topic3Uuid), + mkSet(topic1Uuid, topic2Uuid, topic3Uuid), currentAssignmentForC )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HETEROGENEOUS); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HETEROGENEOUS, + invertedTargetAssignment(members) + ); SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( @@ -345,7 +379,7 @@ public void testReassignmentWhenPartitionsAreAddedForTwoMembers() { members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic3Uuid), + mkSet(topic1Uuid, topic3Uuid), currentAssignmentForA )); @@ -358,14 +392,21 @@ public void testReassignmentWhenPartitionsAreAddedForTwoMembers() { members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid, topic3Uuid, topic4Uuid), + mkSet(topic1Uuid, topic2Uuid, topic3Uuid, topic4Uuid), currentAssignmentForB )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HETEROGENEOUS); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HETEROGENEOUS, + invertedTargetAssignment(members) + ); SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( @@ -406,7 +447,7 @@ public void testReassignmentWhenOneMemberAddedAndPartitionsAddedTwoMembersTwoTop members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic1Uuid), + Collections.singleton(topic1Uuid), currentAssignmentForA )); @@ -417,7 +458,7 @@ public void testReassignmentWhenOneMemberAddedAndPartitionsAddedTwoMembersTwoTop members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), currentAssignmentForB )); @@ -425,14 +466,21 @@ public void testReassignmentWhenOneMemberAddedAndPartitionsAddedTwoMembersTwoTop members.put(memberC, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HETEROGENEOUS); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HETEROGENEOUS, + invertedTargetAssignment(members) + ); SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( @@ -480,7 +528,7 @@ public void testReassignmentWhenOneMemberRemovedAfterInitialAssignmentWithThreeM members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic3Uuid), + mkSet(topic1Uuid, topic3Uuid), currentAssignmentForA )); @@ -490,16 +538,23 @@ public void testReassignmentWhenOneMemberRemovedAfterInitialAssignmentWithThreeM members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic2Uuid), + Collections.singleton(topic2Uuid), currentAssignmentForB )); // Member C was removed - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HETEROGENEOUS); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HETEROGENEOUS, + invertedTargetAssignment(members) + ); SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( @@ -539,7 +594,7 @@ public void testReassignmentWhenOneSubscriptionRemovedAfterInitialAssignmentWith members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic1Uuid), + Collections.singleton(topic1Uuid), currentAssignmentForA )); @@ -550,14 +605,21 @@ public void testReassignmentWhenOneSubscriptionRemovedAfterInitialAssignmentWith members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), currentAssignmentForB )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HETEROGENEOUS); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HETEROGENEOUS, + invertedTargetAssignment(members) + ); SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImplTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImplTest.java new file mode 100644 index 0000000000000..4060b1a4f1b0d --- /dev/null +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImplTest.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.group.assignor; + +import org.apache.kafka.common.Uuid; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + + +public class GroupSpecImplTest { + + private Map members; + private SubscriptionType subscriptionType; + private Map> invertedTargetAssignment; + private GroupSpecImpl groupSpec; + private Uuid topicId; + + @BeforeEach + void setUp() { + members = new HashMap<>(); + + subscriptionType = SubscriptionType.HOMOGENEOUS; + invertedTargetAssignment = new HashMap<>(); + topicId = Uuid.randomUuid(); + + members.put("test-member", new AssignmentMemberSpec( + Optional.empty(), + Optional.empty(), + new HashSet<>(Collections.singletonList(topicId)), + Collections.emptyMap()) + ); + + groupSpec = new GroupSpecImpl( + members, + subscriptionType, + invertedTargetAssignment + ); + } + + @Test + void testMembers() { + assertEquals(members, groupSpec.members()); + } + + @Test + void testSubscriptionType() { + assertEquals(subscriptionType, groupSpec.subscriptionType()); + } + + @Test + void testIsPartitionAssigned() { + Map partitionMap = new HashMap<>(); + partitionMap.put(1, "test-member"); + invertedTargetAssignment.put(topicId, partitionMap); + + assertTrue(groupSpec.isPartitionAssigned(topicId, 1)); + assertFalse(groupSpec.isPartitionAssigned(topicId, 2)); + assertFalse(groupSpec.isPartitionAssigned(Uuid.randomUuid(), 2)); + } +} diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java index 0599c6eaafa80..fdc4f5941fafd 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java @@ -22,7 +22,6 @@ import org.junit.jupiter.api.Test; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -32,9 +31,12 @@ import java.util.Set; import java.util.TreeMap; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.assertAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkOrderedAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.invertedTargetAssignment; import static org.apache.kafka.coordinator.group.CoordinatorRecordHelpersTest.mkMapOfPartitionRacks; import static org.apache.kafka.coordinator.group.assignor.SubscriptionType.HOMOGENEOUS; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -72,13 +74,21 @@ public void testOneMemberNoTopicSubscription() { new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.emptyList(), + Collections.emptySet(), Collections.emptyMap() ) ); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - GroupAssignment groupAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + Collections.emptyMap() + ); + + GroupAssignment groupAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); assertEquals(Collections.emptyMap(), groupAssignment.members()); } @@ -102,15 +112,19 @@ public void testOneMemberSubscribedToNonexistentTopic() { new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic2Uuid), + Collections.singleton(topic2Uuid), Collections.emptyMap() ) ); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + Collections.emptyMap() + ); assertThrows(PartitionAssignorException.class, - () -> assignor.assign(assignmentSpec, subscribedTopicMetadata)); + () -> assignor.assign(groupSpec, subscribedTopicMetadata)); } @Test @@ -133,31 +147,37 @@ public void testFirstAssignmentTwoMembersTwoTopicsNoMemberRacks() { members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic3Uuid), + mkSet(topic1Uuid, topic3Uuid), Collections.emptyMap() )); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic3Uuid), + mkSet(topic1Uuid, topic3Uuid), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); - Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 2), - mkTopicAssignment(topic3Uuid, 1) + mkTopicAssignment(topic1Uuid, 0), + mkTopicAssignment(topic3Uuid, 0, 1) )); expectedAssignment.put(memberB, mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic3Uuid, 0) + mkTopicAssignment(topic1Uuid, 1, 2) )); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + Collections.emptyMap() + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); + + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); + assertAssignment(expectedAssignment, computedAssignment); checkValidityAndBalance(members, computedAssignment); } @@ -176,27 +196,22 @@ public void testFirstAssignmentNumMembersGreaterThanTotalNumPartitions() { members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic3Uuid), + Collections.singleton(topic3Uuid), Collections.emptyMap() )); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic3Uuid), + Collections.singleton(topic3Uuid), Collections.emptyMap() )); members.put(memberC, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic3Uuid), + Collections.singleton(topic3Uuid), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); - // Topic 3 has 2 partitions but three members subscribed to it - one of them should not get an assignment. Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( @@ -209,6 +224,18 @@ public void testFirstAssignmentNumMembersGreaterThanTotalNumPartitions() { Collections.emptyMap() ); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + Collections.emptyMap() + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); + + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); + assertAssignment(expectedAssignment, computedAssignment); checkValidityAndBalance(members, computedAssignment); } @@ -226,22 +253,27 @@ public void testValidityAndBalanceForLargeSampleSet() { )); } - List subscribedTopics = new ArrayList<>(topicMetadata.keySet()); - Map members = new TreeMap<>(); for (int i = 1; i < 50; i++) { members.put("member" + i, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - subscribedTopics, + topicMetadata.keySet(), Collections.emptyMap() )); } - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + Collections.emptyMap() + ); SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); checkValidityAndBalance(members, computedAssignment); } @@ -264,37 +296,26 @@ public void testReassignmentForTwoMembersTwoTopicsGivenUnbalancedPrevAssignment( Map members = new TreeMap<>(); - Map> currentAssignmentForA = new TreeMap<>( - mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 1), - mkTopicAssignment(topic2Uuid, 0, 1) - ) - ); members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), - currentAssignmentForA + mkSet(topic1Uuid, topic2Uuid), + mkOrderedAssignment( + mkTopicAssignment(topic1Uuid, 0, 1), + mkTopicAssignment(topic2Uuid, 0, 1) + ) )); - Map> currentAssignmentForB = new TreeMap<>( - mkAssignment( - mkTopicAssignment(topic1Uuid, 2), - mkTopicAssignment(topic2Uuid, 2) - ) - ); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), - currentAssignmentForB + mkSet(topic1Uuid, topic2Uuid), + mkOrderedAssignment( + mkTopicAssignment(topic1Uuid, 2), + mkTopicAssignment(topic2Uuid, 2) + ) )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); - Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1), @@ -305,6 +326,18 @@ public void testReassignmentForTwoMembersTwoTopicsGivenUnbalancedPrevAssignment( mkTopicAssignment(topic2Uuid, 1, 2) )); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + invertedTargetAssignment(members) + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); + + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); + assertAssignment(expectedAssignment, computedAssignment); checkValidityAndBalance(members, computedAssignment); } @@ -328,47 +361,48 @@ public void testReassignmentWhenPartitionsAreAddedForTwoMembersTwoTopics() { Map members = new TreeMap<>(); - Map> currentAssignmentForA = new TreeMap<>( - mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 2), - mkTopicAssignment(topic2Uuid, 0) - ) - ); members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), - currentAssignmentForA + mkSet(topic1Uuid, topic2Uuid), + mkOrderedAssignment( + mkTopicAssignment(topic1Uuid, 0, 2), + mkTopicAssignment(topic2Uuid, 0) + ) )); - Map> currentAssignmentForB = new TreeMap<>( - mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic2Uuid, 1, 2) - ) - ); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), - currentAssignmentForB + mkSet(topic1Uuid, topic2Uuid), + mkOrderedAssignment( + mkTopicAssignment(topic1Uuid, 1), + mkTopicAssignment(topic2Uuid, 1, 2) + ) )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); - Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 2, 3, 5), - mkTopicAssignment(topic2Uuid, 0, 4) + mkTopicAssignment(topic1Uuid, 0, 2, 3), + mkTopicAssignment(topic2Uuid, 0, 3, 4) )); expectedAssignment.put(memberB, mkAssignment( - mkTopicAssignment(topic1Uuid, 1, 4), - mkTopicAssignment(topic2Uuid, 1, 2, 3) + mkTopicAssignment(topic1Uuid, 1, 4, 5), + mkTopicAssignment(topic2Uuid, 1, 2) )); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + invertedTargetAssignment(members) + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); + + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); + assertAssignment(expectedAssignment, computedAssignment); checkValidityAndBalance(members, computedAssignment); } @@ -391,41 +425,34 @@ public void testReassignmentWhenOneMemberAddedAfterInitialAssignmentWithTwoMembe Map members = new HashMap<>(); - Map> currentAssignmentForA = new TreeMap<>(mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 2), - mkTopicAssignment(topic2Uuid, 0) - )); members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), - currentAssignmentForA + mkSet(topic1Uuid, topic2Uuid), + mkOrderedAssignment( + mkTopicAssignment(topic1Uuid, 0, 2), + mkTopicAssignment(topic2Uuid, 0) + ) )); - Map> currentAssignmentForB = new TreeMap<>(mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic2Uuid, 1, 2) - )); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), - currentAssignmentForB + mkSet(topic1Uuid, topic2Uuid), + mkOrderedAssignment( + mkTopicAssignment(topic1Uuid, 1), + mkTopicAssignment(topic2Uuid, 1, 2) + ) )); // Add a new member to trigger a re-assignment. members.put(memberC, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); - Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 2) @@ -438,6 +465,18 @@ public void testReassignmentWhenOneMemberAddedAfterInitialAssignmentWithTwoMembe mkTopicAssignment(topic2Uuid, 0, 2) )); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + invertedTargetAssignment(members) + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); + + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); + assertAssignment(expectedAssignment, computedAssignment); checkValidityAndBalance(members, computedAssignment); } @@ -460,45 +499,50 @@ public void testReassignmentWhenOneMemberRemovedAfterInitialAssignmentWithThreeM Map members = new HashMap<>(); - Map> currentAssignmentForA = mkAssignment( - mkTopicAssignment(topic1Uuid, 0), - mkTopicAssignment(topic2Uuid, 0) - ); members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), - currentAssignmentForA + mkSet(topic1Uuid, topic2Uuid), + mkAssignment( + mkTopicAssignment(topic1Uuid, 0), + mkTopicAssignment(topic2Uuid, 0) + ) )); - Map> currentAssignmentForB = mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic2Uuid, 1) - ); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), - currentAssignmentForB + mkSet(topic1Uuid, topic2Uuid), + mkAssignment( + mkTopicAssignment(topic1Uuid, 1), + mkTopicAssignment(topic2Uuid, 1) + ) )); // Member C was removed - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); - Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( - mkTopicAssignment(topic1Uuid, 0, 2), - mkTopicAssignment(topic2Uuid, 0) + mkTopicAssignment(topic1Uuid, 0), + mkTopicAssignment(topic2Uuid, 0, 2) )); expectedAssignment.put(memberB, mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic2Uuid, 1, 2) + mkTopicAssignment(topic1Uuid, 1, 2), + mkTopicAssignment(topic2Uuid, 1) )); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + invertedTargetAssignment(members) + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); + + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); + assertAssignment(expectedAssignment, computedAssignment); checkValidityAndBalance(members, computedAssignment); } @@ -522,33 +566,26 @@ public void testReassignmentWhenOneSubscriptionRemovedAfterInitialAssignmentWith // Initial subscriptions were [T1, T2] Map members = new HashMap<>(); - Map> currentAssignmentForA = mkAssignment( - mkTopicAssignment(topic1Uuid, 0), - mkTopicAssignment(topic2Uuid, 0) - ); members.put(memberA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic2Uuid), - currentAssignmentForA + Collections.singleton(topic2Uuid), + mkAssignment( + mkTopicAssignment(topic1Uuid, 0), + mkTopicAssignment(topic2Uuid, 0) + ) )); - Map> currentAssignmentForB = mkAssignment( - mkTopicAssignment(topic1Uuid, 1), - mkTopicAssignment(topic2Uuid, 1) - ); members.put(memberB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic2Uuid), - currentAssignmentForB + Collections.singleton(topic2Uuid), + mkAssignment( + mkTopicAssignment(topic1Uuid, 1), + mkTopicAssignment(topic2Uuid, 1) + ) )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); - Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(memberA, mkAssignment( mkTopicAssignment(topic2Uuid, 0) @@ -557,6 +594,18 @@ public void testReassignmentWhenOneSubscriptionRemovedAfterInitialAssignmentWith mkTopicAssignment(topic2Uuid, 1) )); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + invertedTargetAssignment(members) + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); + + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); + assertAssignment(expectedAssignment, computedAssignment); checkValidityAndBalance(members, computedAssignment); } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java index bafef70ba4b88..7acb9bd5f750c 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.coordinator.group.consumer.TopicMetadata; import org.junit.jupiter.api.Test; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -30,8 +29,10 @@ import java.util.Set; import java.util.TreeMap; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.invertedTargetAssignment; import static org.apache.kafka.coordinator.group.assignor.SubscriptionType.HETEROGENEOUS; import static org.apache.kafka.coordinator.group.assignor.SubscriptionType.HOMOGENEOUS; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -69,13 +70,21 @@ public void testOneConsumerNoTopic() { new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.emptyList(), + Collections.emptySet(), Collections.emptyMap() ) ); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - GroupAssignment groupAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + Collections.emptyMap() + ); + + GroupAssignment groupAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); assertEquals(Collections.emptyMap(), groupAssignment.members()); } @@ -99,15 +108,19 @@ public void testOneConsumerSubscribedToNonExistentTopic() { new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic2Uuid), + Collections.singleton(topic2Uuid), Collections.emptyMap() ) ); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + Collections.emptyMap() + ); assertThrows(PartitionAssignorException.class, - () -> assignor.assign(assignmentSpec, subscribedTopicMetadata)); + () -> assignor.assign(groupSpec, subscribedTopicMetadata)); } @Test @@ -126,34 +139,39 @@ public void testFirstAssignmentTwoConsumersTwoTopicsSameSubscriptions() { createPartitionRacks(2) )); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - Map members = new TreeMap<>(); members.put(consumerA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic3Uuid), + mkSet(topic1Uuid, topic3Uuid), Collections.emptyMap() )); members.put(consumerB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic3Uuid), + mkSet(topic1Uuid, topic3Uuid), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + Collections.emptyMap() + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - Map>> expectedAssignment = new HashMap<>(); + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); + Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(consumerA, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1), mkTopicAssignment(topic3Uuid, 0) )); - expectedAssignment.put(consumerB, mkAssignment( mkTopicAssignment(topic1Uuid, 2), mkTopicAssignment(topic3Uuid, 1) @@ -184,45 +202,49 @@ public void testFirstAssignmentThreeConsumersThreeTopicsDifferentSubscriptions() createPartitionRacks(2) )); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - Map members = new TreeMap<>(); members.put(consumerA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), Collections.emptyMap() )); members.put(consumerB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic3Uuid), + Collections.singleton(topic3Uuid), Collections.emptyMap() )); members.put(consumerC, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic2Uuid, topic3Uuid), + mkSet(topic2Uuid, topic3Uuid), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HETEROGENEOUS); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HETEROGENEOUS, + Collections.emptyMap() + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); + + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); Map>> expectedAssignment = new HashMap<>(); - expectedAssignment.put(consumerA, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1, 2), mkTopicAssignment(topic2Uuid, 0, 1) )); - expectedAssignment.put(consumerB, mkAssignment( mkTopicAssignment(topic3Uuid, 0) )); - expectedAssignment.put(consumerC, mkAssignment( mkTopicAssignment(topic2Uuid, 2), mkTopicAssignment(topic3Uuid, 1) @@ -247,33 +269,40 @@ public void testFirstAssignmentNumConsumersGreaterThanNumPartitions() { createPartitionRacks(2) )); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - Map members = new TreeMap<>(); members.put(consumerA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic3Uuid), + mkSet(topic1Uuid, topic3Uuid), Collections.emptyMap() )); members.put(consumerB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic3Uuid), + mkSet(topic1Uuid, topic3Uuid), Collections.emptyMap() )); members.put(consumerC, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic3Uuid), + mkSet(topic1Uuid, topic3Uuid), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + Collections.emptyMap() + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); + + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); Map>> expectedAssignment = new HashMap<>(); // Topic 3 has 2 partitions but three consumers subscribed to it - one of them will not get a partition. @@ -281,12 +310,10 @@ public void testFirstAssignmentNumConsumersGreaterThanNumPartitions() { mkTopicAssignment(topic1Uuid, 0), mkTopicAssignment(topic3Uuid, 0) )); - expectedAssignment.put(consumerB, mkAssignment( mkTopicAssignment(topic1Uuid, 1), mkTopicAssignment(topic3Uuid, 1) )); - expectedAssignment.put(consumerC, mkAssignment( mkTopicAssignment(topic1Uuid, 2) )); @@ -310,19 +337,16 @@ public void testReassignmentNumConsumersGreaterThanNumPartitionsWhenOneConsumerA createPartitionRacks(2) )); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - Map members = new TreeMap<>(); Map> currentAssignmentForA = mkAssignment( mkTopicAssignment(topic1Uuid, 0), mkTopicAssignment(topic2Uuid, 0) ); - members.put(consumerA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), currentAssignmentForA )); @@ -330,11 +354,10 @@ public void testReassignmentNumConsumersGreaterThanNumPartitionsWhenOneConsumerA mkTopicAssignment(topic1Uuid, 1), mkTopicAssignment(topic2Uuid, 1) ); - members.put(consumerB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), currentAssignmentForB )); @@ -342,20 +365,27 @@ public void testReassignmentNumConsumersGreaterThanNumPartitionsWhenOneConsumerA members.put(consumerC, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + invertedTargetAssignment(members) + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - Map>> expectedAssignment = new HashMap<>(); + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); + Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(consumerA, mkAssignment( mkTopicAssignment(topic1Uuid, 0), mkTopicAssignment(topic2Uuid, 0) )); - expectedAssignment.put(consumerB, mkAssignment( mkTopicAssignment(topic1Uuid, 1), mkTopicAssignment(topic2Uuid, 1) @@ -383,19 +413,16 @@ public void testReassignmentWhenOnePartitionAddedForTwoConsumersTwoTopics() { createPartitionRacks(4) )); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - Map members = new TreeMap<>(); Map> currentAssignmentForA = mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1), mkTopicAssignment(topic2Uuid, 0, 1) ); - members.put(consumerA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), currentAssignmentForA )); @@ -403,24 +430,30 @@ public void testReassignmentWhenOnePartitionAddedForTwoConsumersTwoTopics() { mkTopicAssignment(topic1Uuid, 2), mkTopicAssignment(topic2Uuid, 2) ); - members.put(consumerB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), currentAssignmentForB )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + invertedTargetAssignment(members) + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - Map>> expectedAssignment = new HashMap<>(); + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); + Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(consumerA, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1), mkTopicAssignment(topic2Uuid, 0, 1) )); - expectedAssignment.put(consumerB, mkAssignment( mkTopicAssignment(topic1Uuid, 2, 3), mkTopicAssignment(topic2Uuid, 2, 3) @@ -445,19 +478,16 @@ public void testReassignmentWhenOneConsumerAddedAfterInitialAssignmentWithTwoCon createPartitionRacks(3) )); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - Map members = new TreeMap<>(); Map> currentAssignmentForA = mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1), mkTopicAssignment(topic2Uuid, 0, 1) ); - members.put(consumerA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), currentAssignmentForA )); @@ -465,11 +495,10 @@ public void testReassignmentWhenOneConsumerAddedAfterInitialAssignmentWithTwoCon mkTopicAssignment(topic1Uuid, 2), mkTopicAssignment(topic2Uuid, 2) ); - members.put(consumerB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), currentAssignmentForB )); @@ -477,25 +506,31 @@ public void testReassignmentWhenOneConsumerAddedAfterInitialAssignmentWithTwoCon members.put(consumerC, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + invertedTargetAssignment(members) + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); + + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); Map>> expectedAssignment = new HashMap<>(); - expectedAssignment.put(consumerA, mkAssignment( mkTopicAssignment(topic1Uuid, 0), mkTopicAssignment(topic2Uuid, 0) )); - expectedAssignment.put(consumerB, mkAssignment( mkTopicAssignment(topic1Uuid, 2), mkTopicAssignment(topic2Uuid, 2) )); - expectedAssignment.put(consumerC, mkAssignment( mkTopicAssignment(topic1Uuid, 1), mkTopicAssignment(topic2Uuid, 1) @@ -521,19 +556,16 @@ public void testReassignmentWhenOneConsumerAddedAndOnePartitionAfterInitialAssig createPartitionRacks(3) )); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - Map members = new TreeMap<>(); Map> currentAssignmentForA = mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1), mkTopicAssignment(topic2Uuid, 0, 1) ); - members.put(consumerA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), currentAssignmentForA )); @@ -541,11 +573,10 @@ public void testReassignmentWhenOneConsumerAddedAndOnePartitionAfterInitialAssig mkTopicAssignment(topic1Uuid, 2), mkTopicAssignment(topic2Uuid, 2) ); - members.put(consumerB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), currentAssignmentForB )); @@ -553,25 +584,31 @@ public void testReassignmentWhenOneConsumerAddedAndOnePartitionAfterInitialAssig members.put(consumerC, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic1Uuid), + Collections.singleton(topic1Uuid), Collections.emptyMap() )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HETEROGENEOUS); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HETEROGENEOUS, + invertedTargetAssignment(members) + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - Map>> expectedAssignment = new HashMap<>(); + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); + Map>> expectedAssignment = new HashMap<>(); expectedAssignment.put(consumerA, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1), mkTopicAssignment(topic2Uuid, 0, 1) )); - expectedAssignment.put(consumerB, mkAssignment( mkTopicAssignment(topic1Uuid, 2), mkTopicAssignment(topic2Uuid, 2) )); - expectedAssignment.put(consumerC, mkAssignment( mkTopicAssignment(topic1Uuid, 3) )); @@ -595,8 +632,6 @@ public void testReassignmentWhenOneConsumerRemovedAfterInitialAssignmentWithTwoC createPartitionRacks(3) )); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - Map members = new TreeMap<>(); // Consumer A was removed @@ -604,19 +639,26 @@ public void testReassignmentWhenOneConsumerRemovedAfterInitialAssignmentWithTwoC mkTopicAssignment(topic1Uuid, 2), mkTopicAssignment(topic2Uuid, 2) ); - members.put(consumerB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid), + mkSet(topic1Uuid, topic2Uuid), currentAssignmentForB )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HOMOGENEOUS, + invertedTargetAssignment(members) + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); + + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); Map>> expectedAssignment = new HashMap<>(); - expectedAssignment.put(consumerB, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1, 2), mkTopicAssignment(topic2Uuid, 0, 1, 2) @@ -647,33 +689,28 @@ public void testReassignmentWhenMultipleSubscriptionsRemovedAfterInitialAssignme createPartitionRacks(2) )); - SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); - - Map members = new TreeMap<>(); - // Let initial subscriptions be A -> T1, T2 // B -> T2 // C -> T2, T3 // Change the subscriptions to A -> T1 // B -> T1, T2, T3 // C -> T2 + Map members = new TreeMap<>(); Map> currentAssignmentForA = mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1, 2), mkTopicAssignment(topic2Uuid, 0) ); - members.put(consumerA, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic1Uuid), + Collections.singleton(topic1Uuid), currentAssignmentForA )); Map> currentAssignmentForB = mkAssignment( mkTopicAssignment(topic2Uuid, 1) ); - members.put(consumerB, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Arrays.asList(topic1Uuid, topic2Uuid, topic3Uuid), + mkSet(topic1Uuid, topic2Uuid, topic3Uuid), currentAssignmentForB )); @@ -681,29 +718,34 @@ public void testReassignmentWhenMultipleSubscriptionsRemovedAfterInitialAssignme mkTopicAssignment(topic2Uuid, 2), mkTopicAssignment(topic3Uuid, 0, 1) ); - members.put(consumerC, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonList(topic2Uuid), + Collections.singleton(topic2Uuid), currentAssignmentForC )); - AssignmentSpec assignmentSpec = new AssignmentSpec(members, HETEROGENEOUS); - GroupAssignment computedAssignment = assignor.assign(assignmentSpec, subscribedTopicMetadata); + GroupSpec groupSpec = new GroupSpecImpl( + members, + HETEROGENEOUS, + invertedTargetAssignment(members) + ); + SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadata); + + GroupAssignment computedAssignment = assignor.assign( + groupSpec, + subscribedTopicMetadata + ); Map>> expectedAssignment = new HashMap<>(); - expectedAssignment.put(consumerA, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1) )); - expectedAssignment.put(consumerB, mkAssignment( mkTopicAssignment(topic1Uuid, 2), mkTopicAssignment(topic2Uuid, 0, 1), mkTopicAssignment(topic3Uuid, 0, 1) )); - expectedAssignment.put(consumerC, mkAssignment( mkTopicAssignment(topic2Uuid, 2) )); @@ -711,7 +753,10 @@ public void testReassignmentWhenMultipleSubscriptionsRemovedAfterInitialAssignme assertAssignment(expectedAssignment, computedAssignment); } - private void assertAssignment(Map>> expectedAssignment, GroupAssignment computedGroupAssignment) { + private void assertAssignment( + Map>> expectedAssignment, + GroupAssignment computedGroupAssignment + ) { assertEquals(expectedAssignment.size(), computedGroupAssignment.members().size()); for (String memberId : computedGroupAssignment.members().keySet()) { Map> computedAssignmentForMember = computedGroupAssignment.members().get(memberId).targetPartitions(); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMemberTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMemberTest.java index 1c53445babe57..44ed930f3903e 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMemberTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMemberTest.java @@ -37,6 +37,7 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; import static org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember.classicProtocolListFromJoinRequestProtocolCollection; @@ -75,8 +76,7 @@ public void testNewMember() { assertEquals("rack-id", member.rackId()); assertEquals("client-id", member.clientId()); assertEquals("hostname", member.clientHost()); - // Names are sorted. - assertEquals(Arrays.asList("bar", "foo"), member.subscribedTopicNames()); + assertEquals(mkSet("bar", "foo"), member.subscribedTopicNames()); assertEquals("regex", member.subscribedTopicRegex()); assertEquals("range", member.serverAssignorName().get()); assertEquals(mkAssignment(mkTopicAssignment(topicId1, 1, 2, 3)), member.assignedPartitions()); @@ -175,7 +175,7 @@ public void testUpdateMember() { .maybeUpdateRackId(Optional.of("new-rack-id")) .maybeUpdateInstanceId(Optional.of("new-instance-id")) .maybeUpdateServerAssignorName(Optional.of("new-assignor")) - .maybeUpdateSubscribedTopicNames(Optional.of(Arrays.asList("zar"))) + .maybeUpdateSubscribedTopicNames(Optional.of(Collections.singletonList("zar"))) .maybeUpdateSubscribedTopicRegex(Optional.of("new-regex")) .maybeUpdateRebalanceTimeoutMs(OptionalInt.of(6000)) .build(); @@ -183,7 +183,7 @@ public void testUpdateMember() { assertEquals("new-instance-id", updatedMember.instanceId()); assertEquals("new-rack-id", updatedMember.rackId()); // Names are sorted. - assertEquals(Arrays.asList("zar"), updatedMember.subscribedTopicNames()); + assertEquals(mkSet("zar"), updatedMember.subscribedTopicNames()); assertEquals("new-regex", updatedMember.subscribedTopicRegex()); assertEquals("new-assignor", updatedMember.serverAssignorName().get()); } @@ -210,8 +210,7 @@ public void testUpdateWithConsumerGroupMemberMetadataValue() { assertEquals("rack-id", member.rackId()); assertEquals("client-id", member.clientId()); assertEquals("host-id", member.clientHost()); - // Names are sorted. - assertEquals(Arrays.asList("bar", "foo"), member.subscribedTopicNames()); + assertEquals(mkSet("bar", "foo"), member.subscribedTopicNames()); assertEquals("regex", member.subscribedTopicRegex()); assertEquals("range", member.serverAssignorName().get()); assertEquals( @@ -297,7 +296,7 @@ public void testAsConsumerGroupDescribeMember() { .setInstanceId(instanceId) .setRackId(rackId) .setClientHost(clientHost) - .setSubscribedTopicNames(subscribedTopicNames) + .setSubscribedTopicNames(new ArrayList<>(subscribedTopicNames)) .setSubscribedTopicRegex(subscribedTopicRegex) .setAssignment( new ConsumerGroupDescribeResponseData.Assignment() diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java index 870eb9d7ec862..a483fa3be022c 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java @@ -368,6 +368,42 @@ public void testDeletingMemberRemovesPartitionEpoch() { assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 9)); } + @Test + public void testWaitingOnUnreleasedPartition() { + Uuid fooTopicId = Uuid.randomUuid(); + Uuid barTopicId = Uuid.randomUuid(); + Uuid zarTopicId = Uuid.randomUuid(); + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + + ConsumerGroup consumerGroup = createConsumerGroup("foo"); + consumerGroup.updateTargetAssignment(memberId1, new Assignment(mkAssignment( + mkTopicAssignment(fooTopicId, 1, 2, 3), + mkTopicAssignment(zarTopicId, 7, 8, 9) + ))); + + ConsumerGroupMember member1 = new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(10) + .setState(MemberState.UNRELEASED_PARTITIONS) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 1, 2, 3))) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(barTopicId, 4, 5, 6))) + .build(); + consumerGroup.updateMember(member1); + + assertFalse(consumerGroup.waitingOnUnreleasedPartition(member1)); + + ConsumerGroupMember member2 = new ConsumerGroupMember.Builder(memberId2) + .setMemberEpoch(10) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(zarTopicId, 7))) + .build(); + consumerGroup.updateMember(member2); + + assertTrue(consumerGroup.waitingOnUnreleasedPartition(member1)); + } + @Test public void testGroupState() { Uuid fooTopicId = Uuid.randomUuid(); @@ -729,6 +765,55 @@ public void testUpdateSubscriptionMetadata() { image.cluster() ) ); + + // Compute while taking into account removal of member 1, member 2 and member 3 + assertEquals( + Collections.emptyMap(), + consumerGroup.computeSubscriptionMetadata( + consumerGroup.computeSubscribedTopicNames(new HashSet<>(Arrays.asList(member1, member2, member3))), + image.topics(), + image.cluster() + ) + ); + + // Compute while taking into account removal of member 2 and member 3. + assertEquals( + mkMap( + mkEntry("foo", new TopicMetadata(fooTopicId, "foo", 1, mkMapOfPartitionRacks(1))) + ), + consumerGroup.computeSubscriptionMetadata( + consumerGroup.computeSubscribedTopicNames(new HashSet<>(Arrays.asList(member2, member3))), + image.topics(), + image.cluster() + ) + ); + + // Compute while taking into account removal of member 1. + assertEquals( + mkMap( + mkEntry("bar", new TopicMetadata(barTopicId, "bar", 2, mkMapOfPartitionRacks(2))), + mkEntry("zar", new TopicMetadata(zarTopicId, "zar", 3, mkMapOfPartitionRacks(3))) + ), + consumerGroup.computeSubscriptionMetadata( + consumerGroup.computeSubscribedTopicNames(Collections.singleton(member1)), + image.topics(), + image.cluster() + ) + ); + + // It should return foo, bar and zar. + assertEquals( + mkMap( + mkEntry("foo", new TopicMetadata(fooTopicId, "foo", 1, mkMapOfPartitionRacks(1))), + mkEntry("bar", new TopicMetadata(barTopicId, "bar", 2, mkMapOfPartitionRacks(2))), + mkEntry("zar", new TopicMetadata(zarTopicId, "zar", 3, mkMapOfPartitionRacks(3))) + ), + consumerGroup.computeSubscriptionMetadata( + consumerGroup.computeSubscribedTopicNames(Collections.emptySet()), + image.topics(), + image.cluster() + ) + ); } @Test @@ -798,6 +883,90 @@ public void testUpdateSubscribedTopicNamesAndSubscriptionType() { ); } + @Test + public void testUpdateInvertedAssignment() { + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); + GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class); + ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, "test-group", metricsShard); + Uuid topicId = Uuid.randomUuid(); + String memberId1 = "member1"; + String memberId2 = "member2"; + + // Initial assignment for member1 + Assignment initialAssignment = new Assignment(Collections.singletonMap( + topicId, + new HashSet<>(Collections.singletonList(0)) + )); + consumerGroup.updateTargetAssignment(memberId1, initialAssignment); + + // Verify that partition 0 is assigned to member1. + assertEquals( + mkMap( + mkEntry(topicId, mkMap(mkEntry(0, memberId1))) + ), + consumerGroup.invertedTargetAssignment() + ); + + // New assignment for member1 + Assignment newAssignment = new Assignment(Collections.singletonMap( + topicId, + new HashSet<>(Collections.singletonList(1)) + )); + consumerGroup.updateTargetAssignment(memberId1, newAssignment); + + // Verify that partition 0 is no longer assigned and partition 1 is assigned to member1 + assertEquals( + mkMap( + mkEntry(topicId, mkMap(mkEntry(1, memberId1))) + ), + consumerGroup.invertedTargetAssignment() + ); + + // New assignment for member2 to add partition 1 + Assignment newAssignment2 = new Assignment(Collections.singletonMap( + topicId, + new HashSet<>(Collections.singletonList(1)) + )); + consumerGroup.updateTargetAssignment(memberId2, newAssignment2); + + // Verify that partition 1 is assigned to member2 + assertEquals( + mkMap( + mkEntry(topicId, mkMap(mkEntry(1, memberId2))) + ), + consumerGroup.invertedTargetAssignment() + ); + + // New assignment for member1 to revoke partition 1 and assign partition 0 + Assignment newAssignment1 = new Assignment(Collections.singletonMap( + topicId, + new HashSet<>(Collections.singletonList(0)) + )); + consumerGroup.updateTargetAssignment(memberId1, newAssignment1); + + // Verify that partition 1 is still assigned to member2 and partition 0 is assigned to member1 + assertEquals( + mkMap( + mkEntry(topicId, mkMap( + mkEntry(0, memberId1), + mkEntry(1, memberId2) + )) + ), + consumerGroup.invertedTargetAssignment() + ); + + // Test remove target assignment for member1 + consumerGroup.removeTargetAssignment(memberId1); + + // Verify that partition 0 is no longer assigned and partition 1 is still assigned to member2 + assertEquals( + mkMap( + mkEntry(topicId, mkMap(mkEntry(1, memberId2))) + ), + consumerGroup.invertedTargetAssignment() + ); + } + @Test public void testMetadataRefreshDeadline() { MockTime time = new MockTime(); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java index aff6673193c6c..d5ba038f31895 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TargetAssignmentBuilderTest.java @@ -17,23 +17,26 @@ package org.apache.kafka.coordinator.group.consumer; import org.apache.kafka.common.Uuid; +import org.apache.kafka.coordinator.group.AssignmentTestUtil; +import org.apache.kafka.coordinator.group.MetadataImageBuilder; import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec; -import org.apache.kafka.coordinator.group.assignor.AssignmentSpec; +import org.apache.kafka.coordinator.group.assignor.GroupSpecImpl; import org.apache.kafka.coordinator.group.assignor.SubscriptionType; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; import org.apache.kafka.coordinator.group.assignor.MemberAssignment; import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; +import org.apache.kafka.image.TopicsImage; import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.coordinator.group.Assertions.assertUnorderedListEquals; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; @@ -61,6 +64,7 @@ public static class TargetAssignmentBuilderTestContext { private final Map targetAssignment = new HashMap<>(); private final Map memberAssignments = new HashMap<>(); private final Map staticMembers = new HashMap<>(); + private MetadataImageBuilder topicsImageBuilder = new MetadataImageBuilder(); public TargetAssignmentBuilderTestContext( String groupId, @@ -107,6 +111,8 @@ public Uuid addTopicMetadata( numPartitions, partitionRacks )); + topicsImageBuilder = topicsImageBuilder.addTopic(topicId, topicName, numPartitions); + return topicId; } @@ -156,6 +162,7 @@ public void prepareMemberAssignment( } public TargetAssignmentBuilder.TargetAssignmentResult build() { + TopicsImage topicsImage = topicsImageBuilder.build().topics(); // Prepare expected member specs. Map memberSpecs = new HashMap<>(); @@ -164,7 +171,7 @@ public TargetAssignmentBuilder.TargetAssignmentResult build() { memberSpecs.put(memberId, createAssignmentMemberSpec( member, targetAssignment.getOrDefault(memberId, Assignment.EMPTY), - subscriptionMetadata + topicsImage ) )); @@ -187,7 +194,7 @@ public TargetAssignmentBuilder.TargetAssignmentResult build() { memberSpecs.put(memberId, createAssignmentMemberSpec( updatedMemberOrNull, assignment, - subscriptionMetadata + topicsImage )); } }); @@ -201,8 +208,11 @@ public TargetAssignmentBuilder.TargetAssignmentResult build() { SubscribedTopicMetadata subscribedTopicMetadata = new SubscribedTopicMetadata(topicMetadataMap); SubscriptionType subscriptionType = HOMOGENEOUS; + // Prepare the member assignments per topic partition. + Map> invertedTargetAssignment = AssignmentTestUtil.invertedTargetAssignment(memberSpecs); + // Prepare the expected assignment spec. - AssignmentSpec assignmentSpec = new AssignmentSpec(memberSpecs, subscriptionType); + GroupSpecImpl groupSpec = new GroupSpecImpl(memberSpecs, subscriptionType, invertedTargetAssignment); // We use `any` here to always return an assignment but use `verify` later on // to ensure that the input was correct. @@ -215,7 +225,9 @@ public TargetAssignmentBuilder.TargetAssignmentResult build() { .withStaticMembers(staticMembers) .withSubscriptionMetadata(subscriptionMetadata) .withSubscriptionType(subscriptionType) - .withTargetAssignment(targetAssignment); + .withTargetAssignment(targetAssignment) + .withInvertedTargetAssignment(invertedTargetAssignment) + .withTopicsImage(topicsImage); // Add the updated members or delete the deleted members. updatedMembers.forEach((memberId, updatedMemberOrNull) -> { @@ -232,7 +244,7 @@ public TargetAssignmentBuilder.TargetAssignmentResult build() { // Verify that the assignor was called once with the expected // assignment spec. verify(assignor, times(1)) - .assign(assignmentSpec, subscribedTopicMetadata); + .assign(groupSpec, subscribedTopicMetadata); return result; } @@ -242,6 +254,11 @@ public TargetAssignmentBuilder.TargetAssignmentResult build() { public void testCreateAssignmentMemberSpec() { Uuid fooTopicId = Uuid.randomUuid(); Uuid barTopicId = Uuid.randomUuid(); + TopicsImage topicsImage = new MetadataImageBuilder() + .addTopic(fooTopicId, "foo", 5) + .addTopic(barTopicId, "bar", 5) + .build() + .topics(); ConsumerGroupMember member = new ConsumerGroupMember.Builder("member-id") .setSubscribedTopicNames(Arrays.asList("foo", "bar", "zar")) @@ -249,13 +266,6 @@ public void testCreateAssignmentMemberSpec() { .setInstanceId("instanceId") .build(); - Map subscriptionMetadata = new HashMap() { - { - put("foo", new TopicMetadata(fooTopicId, "foo", 5, Collections.emptyMap())); - put("bar", new TopicMetadata(barTopicId, "bar", 5, Collections.emptyMap())); - } - }; - Assignment assignment = new Assignment(mkAssignment( mkTopicAssignment(fooTopicId, 1, 2, 3), mkTopicAssignment(barTopicId, 1, 2, 3) @@ -264,13 +274,13 @@ public void testCreateAssignmentMemberSpec() { AssignmentMemberSpec assignmentMemberSpec = createAssignmentMemberSpec( member, assignment, - subscriptionMetadata + topicsImage ); assertEquals(new AssignmentMemberSpec( Optional.of("instanceId"), Optional.of("rackId"), - new HashSet<>(Arrays.asList(fooTopicId, barTopicId)), + new TopicIds(mkSet("bar", "foo", "zar"), topicsImage), assignment.partitions() ), assignmentMemberSpec); } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TopicIdsTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TopicIdsTest.java new file mode 100644 index 0000000000000..c937b0551163d --- /dev/null +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TopicIdsTest.java @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.group.consumer; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.coordinator.group.MetadataImageBuilder; +import org.apache.kafka.image.TopicsImage; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TopicIdsTest { + + @Test + public void testTopicNamesCannotBeNull() { + assertThrows(NullPointerException.class, () -> new TopicIds(null, TopicsImage.EMPTY)); + } + + @Test + public void testTopicsImageCannotBeNull() { + assertThrows(NullPointerException.class, () -> new TopicIds(Collections.emptySet(), null)); + } + + @Test + public void testSize() { + Set topicNames = mkSet("foo", "bar", "baz"); + Set topicIds = new TopicIds(topicNames, TopicsImage.EMPTY); + assertEquals(topicNames.size(), topicIds.size()); + } + + @Test + public void testIsEmpty() { + Set topicNames = Collections.emptySet(); + Set topicIds = new TopicIds(topicNames, TopicsImage.EMPTY); + assertEquals(topicNames.size(), topicIds.size()); + } + + @Test + public void testContains() { + Uuid fooUuid = Uuid.randomUuid(); + Uuid barUuid = Uuid.randomUuid(); + Uuid bazUuid = Uuid.randomUuid(); + Uuid quxUuid = Uuid.randomUuid(); + TopicsImage topicsImage = new MetadataImageBuilder() + .addTopic(fooUuid, "foo", 3) + .addTopic(barUuid, "bar", 3) + .addTopic(bazUuid, "qux", 3) + .build() + .topics(); + + Set topicIds = new TopicIds(mkSet("foo", "bar", "baz"), topicsImage); + + assertTrue(topicIds.contains(fooUuid)); + assertTrue(topicIds.contains(barUuid)); + assertFalse(topicIds.contains(bazUuid)); + assertFalse(topicIds.contains(quxUuid)); + } + + @Test + public void testContainsAll() { + Uuid fooUuid = Uuid.randomUuid(); + Uuid barUuid = Uuid.randomUuid(); + Uuid bazUuid = Uuid.randomUuid(); + Uuid quxUuid = Uuid.randomUuid(); + TopicsImage topicsImage = new MetadataImageBuilder() + .addTopic(fooUuid, "foo", 3) + .addTopic(barUuid, "bar", 3) + .addTopic(bazUuid, "baz", 3) + .addTopic(quxUuid, "qux", 3) + .build() + .topics(); + + Set topicIds = new TopicIds(mkSet("foo", "bar", "baz", "qux"), topicsImage); + + assertTrue(topicIds.contains(fooUuid)); + assertTrue(topicIds.contains(barUuid)); + assertTrue(topicIds.contains(bazUuid)); + assertTrue(topicIds.contains(quxUuid)); + assertTrue(topicIds.containsAll(mkSet(fooUuid, barUuid, bazUuid, quxUuid))); + } + + @Test + public void testContainsAllOneTopicConversionFails() { + // topic 'qux' only exists as topic name. + Uuid fooUuid = Uuid.randomUuid(); + Uuid barUuid = Uuid.randomUuid(); + Uuid bazUuid = Uuid.randomUuid(); + Uuid quxUuid = Uuid.randomUuid(); + TopicsImage topicsImage = new MetadataImageBuilder() + .addTopic(fooUuid, "foo", 3) + .addTopic(barUuid, "bar", 3) + .addTopic(bazUuid, "baz", 3) + .build() + .topics(); + + Set topicIds = new TopicIds(mkSet("foo", "bar", "baz", "qux"), topicsImage); + + assertTrue(topicIds.contains(fooUuid)); + assertTrue(topicIds.contains(barUuid)); + assertTrue(topicIds.contains(bazUuid)); + assertTrue(topicIds.containsAll(mkSet(fooUuid, barUuid, bazUuid))); + assertFalse(topicIds.containsAll(mkSet(fooUuid, barUuid, bazUuid, quxUuid))); + } + + @Test + public void testIterator() { + Uuid fooUuid = Uuid.randomUuid(); + Uuid barUuid = Uuid.randomUuid(); + Uuid bazUuid = Uuid.randomUuid(); + Uuid quxUuid = Uuid.randomUuid(); + TopicsImage topicsImage = new MetadataImageBuilder() + .addTopic(fooUuid, "foo", 3) + .addTopic(barUuid, "bar", 3) + .addTopic(bazUuid, "baz", 3) + .addTopic(quxUuid, "qux", 3) + .build() + .topics(); + + Set topicIds = new TopicIds(mkSet("foo", "bar", "baz", "qux"), topicsImage); + Set expectedIds = mkSet(fooUuid, barUuid, bazUuid, quxUuid); + Set actualIds = new HashSet<>(topicIds); + + assertEquals(expectedIds, actualIds); + } + + @Test + public void testIteratorOneTopicConversionFails() { + // topic 'qux' only exists as topic id. + // topic 'quux' only exists as topic name. + Uuid fooUuid = Uuid.randomUuid(); + Uuid barUuid = Uuid.randomUuid(); + Uuid bazUuid = Uuid.randomUuid(); + Uuid qux = Uuid.randomUuid(); + TopicsImage topicsImage = new MetadataImageBuilder() + .addTopic(fooUuid, "foo", 3) + .addTopic(barUuid, "bar", 3) + .addTopic(bazUuid, "baz", 3) + .addTopic(qux, "qux", 3) + .build() + .topics(); + + Set topicIds = new TopicIds(mkSet("foo", "bar", "baz", "quux"), topicsImage); + Set expectedIds = mkSet(fooUuid, barUuid, bazUuid); + Set actualIds = new HashSet<>(topicIds); + + assertEquals(expectedIds, actualIds); + } + + @Test + public void testEquals() { + Uuid topicId = Uuid.randomUuid(); + TopicIds topicIds1 = new TopicIds(Collections.singleton("topic"), + new MetadataImageBuilder() + .addTopic(topicId, "topicId", 3) + .build() + .topics() + ); + + TopicIds topicIds2 = new TopicIds(Collections.singleton("topic"), + new MetadataImageBuilder() + .addTopic(topicId, "topicId", 3) + .build() + .topics() + ); + + assertEquals(topicIds1, topicIds2); + } +} diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java index d874ceae95f00..ae1d404792406 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java @@ -81,6 +81,7 @@ import static org.apache.kafka.coordinator.group.runtime.CoordinatorRuntime.CoordinatorState.FAILED; import static org.apache.kafka.coordinator.group.runtime.CoordinatorRuntime.CoordinatorState.INITIAL; import static org.apache.kafka.coordinator.group.runtime.CoordinatorRuntime.CoordinatorState.LOADING; +import static org.apache.kafka.coordinator.group.runtime.CoordinatorRuntime.MIN_BUFFER_SIZE; import static org.apache.kafka.test.TestUtils.assertFutureThrows; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -2505,6 +2506,57 @@ public void testNonRetryableTimer() throws InterruptedException { assertEquals(0, ctx.timer.size()); } + @Test + public void testTimerScheduleIfAbsent() throws InterruptedException { + MockTimer timer = new MockTimer(); + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(new MockPartitionWriter()) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .build(); + + // Loads the coordinator. + runtime.scheduleLoadOperation(TP, 10); + + // Check initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0, ctx.timer.size()); + + // Timer #1. + AtomicInteger cnt = new AtomicInteger(0); + ctx.timer.scheduleIfAbsent("timer-1", 10, TimeUnit.MILLISECONDS, false, () -> { + cnt.incrementAndGet(); + throw new KafkaException("error"); + }); + + // The coordinator timer should have one pending task. + assertEquals(1, ctx.timer.size()); + + // Advance half of the time to fire the pending timer. + timer.advanceClock(10 / 2); + + // Reschedule timer #1. Since the timer already exists, the timeout shouldn't be refreshed. + ctx.timer.scheduleIfAbsent("timer-1", 10, TimeUnit.MILLISECONDS, false, () -> { + cnt.incrementAndGet(); + throw new KafkaException("error"); + }); + + // Advance the time to fire the pending timer. + timer.advanceClock(10 / 2 + 1); + + // The timer should have been called and the timer should have no pending tasks. + assertEquals(1, cnt.get()); + assertEquals(0, ctx.timer.size()); + } + @Test public void testStateChanges() throws Exception { MockTimer timer = new MockTimer(); @@ -2956,6 +3008,57 @@ public void testCoordinatorCompleteTransactionEventWriteTimeoutTaskIsCancelledWh timer.taskQueue().forEach(taskEntry -> assertTrue(taskEntry.cancelled())); } + @Test + public void testAppendRecordBatchSize() { + MockTimer timer = new MockTimer(); + MockPartitionWriter writer = new MockPartitionWriter(); + StringSerializer serializer = new StringSerializer(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(GroupCoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class)) + .withSerializer(serializer) + .build(); + + // Schedule the loading. + runtime.scheduleLoadOperation(TP, 10); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList()); + + int maxBatchSize = writer.config(TP).maxMessageSize(); + assertTrue(maxBatchSize > MIN_BUFFER_SIZE); + + // Generate enough records to create a batch that has 16KB < batchSize < maxBatchSize + List records = new ArrayList<>(); + for (int i = 0; i < 3000; i++) { + records.add("record-" + i); + } + + // Write #1. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, DEFAULT_WRITE_TIMEOUT, + state -> new CoordinatorResult<>(records, "response1") + ); + + // Verify that the write has not completed exceptionally. + // This will catch any exceptions thrown including RecordTooLargeException. + assertFalse(write1.isCompletedExceptionally()); + + int batchSize = writer.entries(TP).get(0).sizeInBytes(); + assertTrue(batchSize > MIN_BUFFER_SIZE && batchSize < maxBatchSize); + } + private static , U> ArgumentMatcher> coordinatorMatcher( CoordinatorRuntime runtime, TopicPartition tp diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java index 313e46dfc0473..0f2801daec3b9 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/MultiThreadedEventProcessorTest.java @@ -61,8 +61,9 @@ public DelayEventAccumulator(Time time, long takeDelayMs) { @Override public CoordinatorEvent take() { + CoordinatorEvent event = super.take(); time.sleep(takeDelayMs); - return super.take(); + return event; } } @@ -475,9 +476,9 @@ public void testRecordThreadIdleRatio() throws Exception { doAnswer(invocation -> { long threadIdleTime = idleTimeCaptured.getValue(); assertEquals(100, threadIdleTime); - synchronized (recordedIdleTimesMs) { - recordedIdleTimesMs.add(threadIdleTime); - } + + // No synchronization required as the test uses a single event processor thread. + recordedIdleTimesMs.add(threadIdleTime); return null; }).when(mockRuntimeMetrics).recordThreadIdleTime(idleTimeCaptured.capture()); diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/AssignorBenchmarkUtils.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/AssignorBenchmarkUtils.java new file mode 100644 index 0000000000000..2d2edf222f9dd --- /dev/null +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/AssignorBenchmarkUtils.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.jmh.assignor; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.metadata.PartitionRecord; +import org.apache.kafka.common.metadata.TopicRecord; +import org.apache.kafka.coordinator.group.assignor.GroupAssignment; +import org.apache.kafka.coordinator.group.assignor.MemberAssignment; +import org.apache.kafka.image.MetadataDelta; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class AssignorBenchmarkUtils { + /** + * Generate a reverse look up map of partition to member target assignments from the given member spec. + * + * @param groupAssignment The group assignment. + * @return Map of topic partition to member assignments. + */ + public static Map> computeInvertedTargetAssignment( + GroupAssignment groupAssignment + ) { + Map> invertedTargetAssignment = new HashMap<>(); + for (Map.Entry memberEntry : groupAssignment.members().entrySet()) { + String memberId = memberEntry.getKey(); + Map> topicsAndPartitions = memberEntry.getValue().targetPartitions(); + + for (Map.Entry> topicEntry : topicsAndPartitions.entrySet()) { + Uuid topicId = topicEntry.getKey(); + Set partitions = topicEntry.getValue(); + + Map partitionMap = invertedTargetAssignment.computeIfAbsent(topicId, k -> new HashMap<>()); + + for (Integer partitionId : partitions) { + partitionMap.put(partitionId, memberId); + } + } + } + return invertedTargetAssignment; + } + + public static void addTopic( + MetadataDelta delta, + Uuid topicId, + String topicName, + int numPartitions + ) { + // For testing purposes, the following criteria are used: + // - Number of replicas for each partition: 2 + // - Number of brokers available in the cluster: 4 + delta.replay(new TopicRecord().setTopicId(topicId).setName(topicName)); + for (int i = 0; i < numPartitions; i++) { + delta.replay(new PartitionRecord() + .setTopicId(topicId) + .setPartitionId(i) + .setReplicas(Arrays.asList(i % 4, (i + 1) % 4))); + } + } +} diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java index 09ae345dbd383..77a38ab7f4e39 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java @@ -18,16 +18,23 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec; -import org.apache.kafka.coordinator.group.assignor.AssignmentSpec; +import org.apache.kafka.coordinator.group.assignor.GroupSpecImpl; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; import org.apache.kafka.coordinator.group.assignor.MemberAssignment; import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; import org.apache.kafka.coordinator.group.assignor.RangeAssignor; import org.apache.kafka.coordinator.group.assignor.SubscribedTopicDescriber; import org.apache.kafka.coordinator.group.assignor.SubscriptionType; +import org.apache.kafka.coordinator.group.consumer.TopicIds; import org.apache.kafka.coordinator.group.assignor.UniformAssignor; import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata; import org.apache.kafka.coordinator.group.consumer.TopicMetadata; + +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.MetadataProvenance; +import org.apache.kafka.image.TopicsImage; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -44,7 +51,6 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -115,11 +121,13 @@ public enum AssignmentType { private static final int MAX_BUCKET_COUNT = 5; - private AssignmentSpec assignmentSpec; + private GroupSpecImpl groupSpec; private SubscribedTopicDescriber subscribedTopicDescriber; - private final List allTopicIds = new ArrayList<>(); + private final List allTopicNames = new ArrayList<>(); + + private TopicsImage topicsImage = TopicsImage.EMPTY; @Setup(Level.Trial) public void setup() { @@ -136,6 +144,7 @@ public void setup() { } private Map createTopicMetadata() { + MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY); Map topicMetadata = new HashMap<>(); int partitionsPerTopicCount = (memberCount * partitionsToMemberRatio) / topicCount; @@ -146,15 +155,23 @@ private Map createTopicMetadata() { for (int i = 0; i < topicCount; i++) { Uuid topicUuid = Uuid.randomUuid(); String topicName = "topic" + i; - allTopicIds.add(topicUuid); + allTopicNames.add(topicName); topicMetadata.put(topicUuid, new TopicMetadata( topicUuid, topicName, partitionsPerTopicCount, partitionRacks )); + + AssignorBenchmarkUtils.addTopic( + delta, + topicUuid, + topicName, + partitionsPerTopicCount + ); } + topicsImage = delta.apply(MetadataProvenance.EMPTY).topics(); return topicMetadata; } @@ -167,7 +184,7 @@ private void createAssignmentSpec() { if (subscriptionType == HOMOGENEOUS) { for (int i = 0; i < numberOfMembers; i++) { - addMemberSpec(members, i, new HashSet<>(allTopicIds)); + addMemberSpec(members, i, new TopicIds(new HashSet<>(allTopicNames), topicsImage)); } } else { // Adjust bucket count based on member count when member count < max bucket count. @@ -189,7 +206,7 @@ private void createAssignmentSpec() { int topicStartIndex = bucket * bucketSizeTopics; int topicEndIndex = Math.min((bucket + 1) * bucketSizeTopics, topicCount); - Set bucketTopics = new HashSet<>(allTopicIds.subList(topicStartIndex, topicEndIndex)); + TopicIds bucketTopics = new TopicIds(new HashSet<>(allTopicNames.subList(topicStartIndex, topicEndIndex)), topicsImage); // Assign topics to each member in the current bucket for (int i = memberStartIndex; i < memberEndIndex; i++) { @@ -198,7 +215,7 @@ private void createAssignmentSpec() { } } - this.assignmentSpec = new AssignmentSpec(members, subscriptionType); + this.groupSpec = new GroupSpecImpl(members, subscriptionType, Collections.emptyMap()); } private Optional rackId(int memberIndex) { @@ -234,25 +251,32 @@ private static Map> mkMapOfPartitionRacks(int numPartitions } private void simulateIncrementalRebalance() { - GroupAssignment initialAssignment = partitionAssignor.assign(assignmentSpec, subscribedTopicDescriber); + GroupAssignment initialAssignment = partitionAssignor.assign(groupSpec, subscribedTopicDescriber); Map members = initialAssignment.members(); + Map> invertedTargetAssignment = AssignorBenchmarkUtils.computeInvertedTargetAssignment(initialAssignment); + Map updatedMembers = new HashMap<>(); - members.forEach((memberId, memberAssignment) -> { - AssignmentMemberSpec memberSpec = assignmentSpec.members().get(memberId); + + groupSpec.members().forEach((memberId, assignmentMemberSpec) -> { + MemberAssignment memberAssignment = members.getOrDefault( + memberId, + new MemberAssignment(Collections.emptyMap()) + ); + updatedMembers.put(memberId, new AssignmentMemberSpec( - memberSpec.instanceId(), - memberSpec.rackId(), - memberSpec.subscribedTopicIds(), - memberAssignment.targetPartitions() + assignmentMemberSpec.instanceId(), + assignmentMemberSpec.rackId(), + assignmentMemberSpec.subscribedTopicIds(), + Collections.unmodifiableMap(memberAssignment.targetPartitions()) )); }); - Collection subscribedTopicIdsForNewMember; + Set subscribedTopicIdsForNewMember; if (subscriptionType == HETEROGENEOUS) { subscribedTopicIdsForNewMember = updatedMembers.get("member" + (memberCount - 2)).subscribedTopicIds(); } else { - subscribedTopicIdsForNewMember = allTopicIds; + subscribedTopicIdsForNewMember = new TopicIds(new HashSet<>(allTopicNames), topicsImage); } Optional rackId = rackId(memberCount - 1); @@ -263,13 +287,13 @@ private void simulateIncrementalRebalance() { Collections.emptyMap() )); - assignmentSpec = new AssignmentSpec(updatedMembers, subscriptionType); + groupSpec = new GroupSpecImpl(updatedMembers, subscriptionType, invertedTargetAssignment); } @Benchmark @Threads(1) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void doAssignment() { - partitionAssignor.assign(assignmentSpec, subscribedTopicDescriber); + partitionAssignor.assign(groupSpec, subscribedTopicDescriber); } } diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java index 79a64cb667aab..7d67f07d3e662 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java @@ -18,10 +18,11 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec; -import org.apache.kafka.coordinator.group.assignor.AssignmentSpec; +import org.apache.kafka.coordinator.group.assignor.GroupSpecImpl; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; import org.apache.kafka.coordinator.group.assignor.MemberAssignment; import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; +import org.apache.kafka.coordinator.group.consumer.TopicIds; import org.apache.kafka.coordinator.group.assignor.UniformAssignor; import org.apache.kafka.coordinator.group.consumer.Assignment; import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember; @@ -29,6 +30,11 @@ import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder; import org.apache.kafka.coordinator.group.consumer.TopicMetadata; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.MetadataProvenance; +import org.apache.kafka.image.TopicsImage; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -46,6 +52,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -81,11 +88,13 @@ public class TargetAssignmentBuilderBenchmark { private TargetAssignmentBuilder targetAssignmentBuilder; - private AssignmentSpec assignmentSpec; + private GroupSpecImpl groupSpec; + + private Map> invertedTargetAssignment; private final List allTopicNames = new ArrayList<>(); - private final List allTopicIds = new ArrayList<>(); + private TopicsImage topicsImage; @Setup(Level.Trial) public void setup() { @@ -95,7 +104,7 @@ public void setup() { subscriptionMetadata = generateMockSubscriptionMetadata(); Map members = generateMockMembers(); - Map existingTargetAssignment = generateMockInitialTargetAssignment(); + Map existingTargetAssignment = generateMockInitialTargetAssignmentAndUpdateInvertedTargetAssignment(); ConsumerGroupMember newMember = new ConsumerGroupMember.Builder("newMember") .setSubscribedTopicNames(allTopicNames) @@ -104,8 +113,10 @@ public void setup() { targetAssignmentBuilder = new TargetAssignmentBuilder(GROUP_ID, GROUP_EPOCH, partitionAssignor) .withMembers(members) .withSubscriptionMetadata(subscriptionMetadata) - .withTargetAssignment(existingTargetAssignment) .withSubscriptionType(HOMOGENEOUS) + .withTargetAssignment(existingTargetAssignment) + .withInvertedTargetAssignment(invertedTargetAssignment) + .withTopicsImage(topicsImage) .addOrUpdateMember(newMember.memberId(), newMember); } @@ -123,13 +134,13 @@ private Map generateMockMembers() { private Map generateMockSubscriptionMetadata() { Map subscriptionMetadata = new HashMap<>(); + MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY); int partitionsPerTopicCount = (memberCount * partitionsToMemberRatio) / topicCount; for (int i = 0; i < topicCount; i++) { String topicName = "topic-" + i; Uuid topicId = Uuid.randomUuid(); allTopicNames.add(topicName); - allTopicIds.add(topicId); TopicMetadata metadata = new TopicMetadata( topicId, @@ -138,12 +149,20 @@ private Map generateMockSubscriptionMetadata() { Collections.emptyMap() ); subscriptionMetadata.put(topicName, metadata); + + AssignorBenchmarkUtils.addTopic( + delta, + topicId, + topicName, + partitionsPerTopicCount + ); } + topicsImage = delta.apply(MetadataProvenance.EMPTY).topics(); return subscriptionMetadata; } - private Map generateMockInitialTargetAssignment() { + private Map generateMockInitialTargetAssignmentAndUpdateInvertedTargetAssignment() { Map topicMetadataMap = new HashMap<>(topicCount); subscriptionMetadata.forEach((topicName, topicMetadata) -> topicMetadataMap.put( @@ -155,19 +174,17 @@ private Map generateMockInitialTargetAssignment() { createAssignmentSpec(); GroupAssignment groupAssignment = partitionAssignor.assign( - assignmentSpec, + groupSpec, new SubscribedTopicMetadata(topicMetadataMap) ); + invertedTargetAssignment = AssignorBenchmarkUtils.computeInvertedTargetAssignment(groupAssignment); Map initialTargetAssignment = new HashMap<>(memberCount); for (Map.Entry entry : groupAssignment.members().entrySet()) { String memberId = entry.getKey(); Map> topicPartitions = entry.getValue().targetPartitions(); - - Assignment assignment = new Assignment(topicPartitions); - - initialTargetAssignment.put(memberId, assignment); + initialTargetAssignment.put(memberId, new Assignment(topicPartitions)); } return initialTargetAssignment; @@ -182,11 +199,11 @@ private void createAssignmentSpec() { members.put(memberId, new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - allTopicIds, + new TopicIds(new HashSet<>(allTopicNames), topicsImage), Collections.emptyMap() )); } - assignmentSpec = new AssignmentSpec(members, HOMOGENEOUS); + groupSpec = new GroupSpecImpl(members, HOMOGENEOUS, Collections.emptyMap()); } @Benchmark diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/ValuesBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/ValuesBenchmark.java new file mode 100644 index 0000000000000..ddcdb5e2418ff --- /dev/null +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/ValuesBenchmark.java @@ -0,0 +1,297 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.jmh.connect; + +import org.apache.kafka.connect.data.Date; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.data.Time; +import org.apache.kafka.connect.data.Timestamp; +import org.apache.kafka.connect.data.Values; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.BiFunction; +import java.util.function.Function; + +/** + * This benchmark tests the performance of the {@link Values} data handling class. + */ +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 3) +@Measurement(iterations = 7) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +public class ValuesBenchmark { + + private static final Schema MAP_INT_STRING_SCHEMA = SchemaBuilder.map(Schema.INT32_SCHEMA, Schema.STRING_SCHEMA).build(); + private static final Schema FLAT_STRUCT_SCHEMA = SchemaBuilder.struct() + .field("field", Schema.INT32_SCHEMA) + .build(); + private static final Schema STRUCT_SCHEMA = SchemaBuilder.struct() + .field("first", Schema.INT32_SCHEMA) + .field("second", Schema.STRING_SCHEMA) + .field("array", SchemaBuilder.array(Schema.INT32_SCHEMA).build()) + .field("map", MAP_INT_STRING_SCHEMA) + .field("nested", FLAT_STRUCT_SCHEMA) + .build(); + private static final SchemaAndValue[] TEST_VALUES = { + SchemaAndValue.NULL, + new SchemaAndValue(Schema.OPTIONAL_BOOLEAN_SCHEMA, null), + new SchemaAndValue(Schema.BOOLEAN_SCHEMA, true), + new SchemaAndValue(Schema.BOOLEAN_SCHEMA, false), + new SchemaAndValue(Schema.OPTIONAL_INT8_SCHEMA, null), + new SchemaAndValue(Schema.INT8_SCHEMA, (byte) 0), + new SchemaAndValue(Schema.INT8_SCHEMA, Byte.MAX_VALUE), + new SchemaAndValue(Schema.OPTIONAL_INT16_SCHEMA, null), + new SchemaAndValue(Schema.INT16_SCHEMA, (short) 0), + new SchemaAndValue(Schema.INT16_SCHEMA, Short.MAX_VALUE), + new SchemaAndValue(Schema.OPTIONAL_INT32_SCHEMA, null), + new SchemaAndValue(Schema.INT32_SCHEMA, 0), + new SchemaAndValue(Schema.INT32_SCHEMA, Integer.MAX_VALUE), + new SchemaAndValue(Schema.OPTIONAL_INT64_SCHEMA, null), + new SchemaAndValue(Schema.INT64_SCHEMA, (long) 0), + new SchemaAndValue(Schema.INT64_SCHEMA, Long.MAX_VALUE), + new SchemaAndValue(Schema.OPTIONAL_FLOAT32_SCHEMA, null), + new SchemaAndValue(Schema.FLOAT32_SCHEMA, (float) 0), + new SchemaAndValue(Schema.FLOAT32_SCHEMA, 0.1f), + new SchemaAndValue(Schema.FLOAT64_SCHEMA, 1.1f), + new SchemaAndValue(Schema.FLOAT32_SCHEMA, Float.MAX_VALUE), + new SchemaAndValue(Schema.OPTIONAL_FLOAT64_SCHEMA, null), + new SchemaAndValue(Schema.FLOAT64_SCHEMA, (double) 0), + new SchemaAndValue(Schema.FLOAT64_SCHEMA, (double) 0.1f), + new SchemaAndValue(Schema.FLOAT64_SCHEMA, (double) 1.1f), + new SchemaAndValue(Schema.FLOAT64_SCHEMA, Double.MAX_VALUE), + new SchemaAndValue(Schema.OPTIONAL_STRING_SCHEMA, null), + new SchemaAndValue(Date.SCHEMA, "2019-08-23"), + new SchemaAndValue(Time.SCHEMA, "14:34:54.346Z"), + new SchemaAndValue(Timestamp.SCHEMA, "2019-08-23T14:34:54.346Z"), + new SchemaAndValue(Schema.STRING_SCHEMA, ""), + new SchemaAndValue(Schema.STRING_SCHEMA, "a-random-string"), + new SchemaAndValue(Schema.STRING_SCHEMA, "[]"), + new SchemaAndValue(Schema.STRING_SCHEMA, "[1, 2, 3]"), + new SchemaAndValue(Schema.STRING_SCHEMA, "{}"), + new SchemaAndValue(Schema.STRING_SCHEMA, "{\"1\": 2, \"3\": 4}"), + new SchemaAndValue(SchemaBuilder.array(Schema.INT16_SCHEMA), new short[]{1, 2, 3}), + new SchemaAndValue(SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.BOOLEAN_SCHEMA), + Collections.singletonMap("key", true)), + new SchemaAndValue(STRUCT_SCHEMA, new Struct(STRUCT_SCHEMA) + .put("first", 1) + .put("second", "foo") + .put("array", Arrays.asList(1, 2, 3)) + .put("map", Collections.singletonMap(1, "value")) + .put("nested", new Struct(FLAT_STRUCT_SCHEMA).put("field", 12))), + }; + + private SchemaAndValue[] convertToBooleanCases; + private SchemaAndValue[] convertToByteCases; + private SchemaAndValue[] convertToDateCases; + private SchemaAndValue[] convertToDecimalCases; + private SchemaAndValue[] convertToDoubleCases; + private SchemaAndValue[] convertToFloatCases; + private SchemaAndValue[] convertToShortCases; + private SchemaAndValue[] convertToListCases; + private SchemaAndValue[] convertToMapCases; + private SchemaAndValue[] convertToLongCases; + private SchemaAndValue[] convertToIntegerCases; + private SchemaAndValue[] convertToStructCases; + private SchemaAndValue[] convertToTimeCases; + private SchemaAndValue[] convertToTimestampCases; + private SchemaAndValue[] convertToStringCases; + private String[] parseStringCases; + + private SchemaAndValue[] successfulCases(BiFunction fn) { + List successful = new ArrayList<>(); + for (SchemaAndValue testCase : TEST_VALUES) { + try { + fn.apply(testCase.schema(), testCase.value()); + successful.add(testCase); + } catch (Throwable ignored) { + } + } + return successful.toArray(new SchemaAndValue[]{}); + } + + private String[] casesToString(Function fn) { + List successful = new ArrayList<>(); + for (SchemaAndValue testCase : TEST_VALUES) { + String v = String.valueOf(testCase.value()); + try { + fn.apply(v); + successful.add(v); + } catch (Throwable ignored) { + } + } + return successful.toArray(new String[]{}); + } + + @Setup + public void setup() { + convertToBooleanCases = successfulCases(Values::convertToBoolean); + convertToByteCases = successfulCases(Values::convertToByte); + convertToDateCases = successfulCases(Values::convertToDate); + convertToDecimalCases = successfulCases((schema, object) -> Values.convertToDecimal(schema, object, 1)); + convertToDoubleCases = successfulCases(Values::convertToDouble); + convertToFloatCases = successfulCases(Values::convertToFloat); + convertToShortCases = successfulCases(Values::convertToShort); + convertToListCases = successfulCases(Values::convertToList); + convertToMapCases = successfulCases(Values::convertToMap); + convertToLongCases = successfulCases(Values::convertToLong); + convertToIntegerCases = successfulCases(Values::convertToInteger); + convertToStructCases = successfulCases(Values::convertToStruct); + convertToTimeCases = successfulCases(Values::convertToTime); + convertToTimestampCases = successfulCases(Values::convertToTimestamp); + convertToStringCases = successfulCases(Values::convertToString); + parseStringCases = casesToString(Values::parseString); + } + + @Benchmark + public void testConvertToBoolean(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToBooleanCases) { + blackhole.consume(Values.convertToBoolean(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testConvertToByte(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToByteCases) { + blackhole.consume(Values.convertToByte(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testConvertToDate(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToDateCases) { + blackhole.consume(Values.convertToDate(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testConvertToDecimal(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToDecimalCases) { + blackhole.consume(Values.convertToDecimal(testCase.schema(), testCase.value(), 1)); + } + } + + @Benchmark + public void testConvertToDouble(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToDoubleCases) { + blackhole.consume(Values.convertToDouble(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testConvertToFloat(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToFloatCases) { + blackhole.consume(Values.convertToFloat(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testConvertToShort(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToShortCases) { + blackhole.consume(Values.convertToShort(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testConvertToList(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToListCases) { + blackhole.consume(Values.convertToList(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testConvertToMap(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToMapCases) { + blackhole.consume(Values.convertToMap(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testConvertToLong(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToLongCases) { + blackhole.consume(Values.convertToLong(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testConvertToInteger(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToIntegerCases) { + blackhole.consume(Values.convertToInteger(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testConvertToStruct(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToStructCases) { + blackhole.consume(Values.convertToStruct(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testConvertToTime(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToTimeCases) { + blackhole.consume(Values.convertToTime(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testConvertToTimestamp(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToTimestampCases) { + blackhole.consume(Values.convertToTimestamp(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testConvertToString(Blackhole blackhole) { + for (SchemaAndValue testCase : convertToStringCases) { + blackhole.consume(Values.convertToString(testCase.schema(), testCase.value())); + } + } + + @Benchmark + public void testInferSchema(Blackhole blackhole) { + for (SchemaAndValue testCase : TEST_VALUES) { + blackhole.consume(Values.inferSchema(testCase.value())); + } + } + + @Benchmark + public void testParseString(Blackhole blackhole) { + for (String testCase : parseStringCases) { + blackhole.consume(Values.parseString(testCase)); + } + } +} diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java index ecd644b56dd3a..c1c36447c2940 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java @@ -95,7 +95,6 @@ import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; -import java.util.Optional; import java.util.Properties; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -112,7 +111,7 @@ public class ReplicaFetcherThreadBenchmark { private final File logDir = new File(System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString()); private final KafkaScheduler scheduler = new KafkaScheduler(1, true, "scheduler"); - private final Pool pool = new Pool(Option.empty()); + private final Pool pool = new Pool<>(Option.empty()); private final Metrics metrics = new Metrics(); private final Option topicId = Option.apply(Uuid.randomUuid()); @Param({"100", "500", "1000", "5000"}) @@ -133,7 +132,7 @@ public void setup() throws IOException { KafkaConfig config = new KafkaConfig(props); LogConfig logConfig = createLogConfig(); - BrokerTopicStats brokerTopicStats = new BrokerTopicStats(Optional.empty()); + BrokerTopicStats brokerTopicStats = new BrokerTopicStats(false); LogDirFailureChannel logDirFailureChannel = Mockito.mock(LogDirFailureChannel.class); List logDirs = Collections.singletonList(logDir); logManager = new LogManagerBuilder(). @@ -217,7 +216,7 @@ public RecordsSend toSend() { // TODO: fix to support raft ZkMetadataCache metadataCache = MetadataCache.zkMetadataCache(0, - config.interBrokerProtocolVersion(), BrokerFeatures.createEmpty(), null, false); + config.interBrokerProtocolVersion(), BrokerFeatures.createEmpty(), false); metadataCache.updateMetadata(0, updateMetadataRequest); replicaManager = new ReplicaManagerBuilder(). diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/KRaftMetadataRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/KRaftMetadataRequestBenchmark.java index 2f3a0a68c2e31..19b2f25d5a4b8 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/KRaftMetadataRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/KRaftMetadataRequestBenchmark.java @@ -59,7 +59,7 @@ import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.MetadataProvenance; import org.apache.kafka.raft.QuorumConfig; -import org.apache.kafka.server.common.Features; +import org.apache.kafka.server.common.FinalizedFeatures; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.config.KRaftConfigs; @@ -114,7 +114,7 @@ public class KRaftMetadataRequestBenchmark { clientQuotaManager, clientRequestQuotaManager, controllerMutationQuotaManager, replicaQuotaManager, replicaQuotaManager, replicaQuotaManager, Option.empty()); private final FetchManager fetchManager = Mockito.mock(FetchManager.class); - private final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(Optional.empty()); + private final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(false); private final KafkaPrincipal principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "test-user"); @Param({"500", "1000", "5000"}) private int topicCount; @@ -204,7 +204,7 @@ private KafkaApis createKafkaApis() { ApiMessageType.ListenerType.BROKER, false, false, - () -> Features.fromKRaftVersion(MetadataVersion.latestTesting()))). + () -> FinalizedFeatures.fromKRaftVersion(MetadataVersion.latestTesting()))). build(); } diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java index 1c5e499145fcf..5d3591454bd50 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java @@ -31,7 +31,6 @@ import kafka.server.FetchManager; import kafka.server.KafkaApis; import kafka.server.KafkaConfig; -import kafka.server.KafkaConfig$; import kafka.server.MetadataCache; import kafka.server.QuotaFactory; import kafka.server.ReplicaManager; @@ -60,8 +59,9 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.utils.Time; import org.apache.kafka.coordinator.group.GroupCoordinator; -import org.apache.kafka.server.common.Features; +import org.apache.kafka.server.common.FinalizedFeatures; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.config.ZkConfigs; import org.mockito.Mockito; import org.openjdk.jmh.annotations.Benchmark; @@ -109,7 +109,7 @@ public class MetadataRequestBenchmark { private final Metrics metrics = new Metrics(); private final int brokerId = 1; private final ZkMetadataCache metadataCache = MetadataCache.zkMetadataCache(brokerId, - MetadataVersion.latestTesting(), BrokerFeatures.createEmpty(), null, false); + MetadataVersion.latestTesting(), BrokerFeatures.createEmpty(), false); private final ClientQuotaManager clientQuotaManager = Mockito.mock(ClientQuotaManager.class); private final ClientRequestQuotaManager clientRequestQuotaManager = Mockito.mock(ClientRequestQuotaManager.class); private final ControllerMutationQuotaManager controllerMutationQuotaManager = Mockito.mock(ControllerMutationQuotaManager.class); @@ -118,7 +118,7 @@ public class MetadataRequestBenchmark { clientQuotaManager, clientRequestQuotaManager, controllerMutationQuotaManager, replicaQuotaManager, replicaQuotaManager, replicaQuotaManager, Option.empty()); private final FetchManager fetchManager = Mockito.mock(FetchManager.class); - private final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(Optional.empty()); + private final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(false); private final KafkaPrincipal principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "test-user"); @Param({"500", "1000", "5000"}) private int topicCount; @@ -178,7 +178,7 @@ private List endpoints(final int brokerId) { private KafkaApis createKafkaApis() { Properties kafkaProps = new Properties(); kafkaProps.put(ZkConfigs.ZK_CONNECT_CONFIG, "zk"); - kafkaProps.put(KafkaConfig$.MODULE$.BrokerIdProp(), brokerId + ""); + kafkaProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId + ""); KafkaConfig config = new KafkaConfig(kafkaProps); return new KafkaApisBuilder(). setRequestChannel(requestChannel). @@ -204,7 +204,7 @@ private KafkaApis createKafkaApis() { ApiMessageType.ListenerType.ZK_BROKER, false, false, - () -> Features.fromKRaftVersion(MetadataVersion.latestTesting()))). + () -> FinalizedFeatures.fromKRaftVersion(MetadataVersion.latestTesting()))). build(); } diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java index 8d016b504e8c7..62840ef9b86a0 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java @@ -94,7 +94,7 @@ public void setup() throws IOException { scheduler.startup(); LogConfig logConfig = new LogConfig(new Properties()); - BrokerTopicStats brokerTopicStats = new BrokerTopicStats(Optional.empty()); + BrokerTopicStats brokerTopicStats = new BrokerTopicStats(false); LogDirFailureChannel logDirFailureChannel = Mockito.mock(LogDirFailureChannel.class); logManager = new LogManagerBuilder(). setLogDirs(Collections.singletonList(logDir)). diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java index 7495dd37d2a5b..686ee7acf67da 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java @@ -74,7 +74,7 @@ public class UpdateFollowerFetchStateBenchmark { private final Option topicId = OptionConverters.toScala(Optional.of(Uuid.randomUuid())); private final File logDir = new File(System.getProperty("java.io.tmpdir"), topicPartition.toString()); private final KafkaScheduler scheduler = new KafkaScheduler(1, true, "scheduler"); - private final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(Optional.empty()); + private final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(false); private final LogDirFailureChannel logDirFailureChannel = Mockito.mock(LogDirFailureChannel.class); private long nextOffset = 0; private LogManager logManager; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java index 0a9c5ff5d4424..1190854a339e0 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java @@ -36,7 +36,6 @@ import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Optional; import java.util.Random; import java.util.stream.IntStream; @@ -80,7 +79,7 @@ public enum Bytes { ByteBuffer[] batchBuffers; RequestLocal requestLocal; LogValidator.MetricsRecorder validatorMetricsRecorder = UnifiedLog.newValidatorMetricsRecorder( - new BrokerTopicStats(Optional.empty()).allTopicsStats()); + new BrokerTopicStats(false).allTopicsStats()); @Setup public void init() { diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java index fb76bab02cfa1..059e41dadec9d 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java @@ -57,7 +57,6 @@ import java.io.File; import java.util.ArrayList; import java.util.List; -import java.util.Optional; import java.util.Properties; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -112,11 +111,11 @@ public void setup() { new LogConfig(new Properties()), new MockConfigRepository(), new CleanerConfig(1, 4 * 1024 * 1024L, 0.9d, 1024 * 1024, 32 * 1024 * 1024, Double.MAX_VALUE, 15 * 1000, true), time, MetadataVersion.latestTesting(), 4, false, Option.empty(), false, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT); scheduler.startup(); - final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(Optional.empty()); + final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(false); final MetadataCache metadataCache = MetadataCache.zkMetadataCache(this.brokerProperties.brokerId(), this.brokerProperties.interBrokerProtocolVersion(), - BrokerFeatures.createEmpty(), null, false); + BrokerFeatures.createEmpty(), false); this.quotaManagers = QuotaFactory.instantiate(this.brokerProperties, this.metrics, diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java index 86db11d4e36b4..24261ba1071b6 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java @@ -63,7 +63,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.Properties; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -114,7 +113,7 @@ public void setup() { this.metrics = new Metrics(); this.time = Time.SYSTEM; this.failureChannel = new LogDirFailureChannel(brokerProperties.logDirs().size()); - final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(Optional.empty()); + final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(false); final List files = JavaConverters.seqAsJavaList(brokerProperties.logDirs()).stream().map(File::new).collect(Collectors.toList()); CleanerConfig cleanerConfig = new CleanerConfig(1, @@ -162,7 +161,7 @@ public Properties getEntityConfigs(String rootEntityType, String sanitizedEntity setBrokerTopicStats(brokerTopicStats). setMetadataCache(MetadataCache.zkMetadataCache(this.brokerProperties.brokerId(), this.brokerProperties.interBrokerProtocolVersion(), BrokerFeatures.createEmpty(), - null, false)). + false)). setLogDirFailureChannel(failureChannel). setAlterPartitionManager(alterPartitionManager). build(); diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/storage/ProducerStateManagerBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/storage/ProducerStateManagerBench.java index 291c78d72adb0..49744e0b29453 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/storage/ProducerStateManagerBench.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/storage/ProducerStateManagerBench.java @@ -49,7 +49,7 @@ @Fork(1) @BenchmarkMode(Mode.AverageTime) @State(value = Scope.Benchmark) -@OutputTimeUnit(TimeUnit.NANOSECONDS) +@OutputTimeUnit(TimeUnit.MICROSECONDS) public class ProducerStateManagerBench { Time time = new MockTime(); final int producerIdExpirationMs = 1000; @@ -57,10 +57,10 @@ public class ProducerStateManagerBench { ProducerStateManager manager; Path tempDirectory; - @Param({"100", "1000", "10000", "100000"}) + @Param({"10000", "100000", "1000000"}) public int numProducerIds; - @Setup(Level.Trial) + @Setup(Level.Invocation) public void setup() throws IOException { tempDirectory = Files.createTempDirectory("kafka-logs"); manager = new ProducerStateManager( @@ -70,17 +70,6 @@ public void setup() throws IOException { new ProducerStateManagerConfig(producerIdExpirationMs, false), time ); - } - - - @TearDown(Level.Trial) - public void tearDown() throws Exception { - Files.deleteIfExists(tempDirectory); - } - - @Benchmark - @Threads(1) - public void testDeleteExpiringIds() { short epoch = 0; for (long i = 0L; i < numProducerIds; i++) { final ProducerStateEntry entry = new ProducerStateEntry( @@ -93,7 +82,17 @@ public void testDeleteExpiringIds() { ); manager.loadProducerEntry(entry); } + } + + @TearDown(Level.Invocation) + public void tearDown() throws Exception { + Files.deleteIfExists(tempDirectory); + } + + @Benchmark + @Threads(1) + public void testDeleteExpiringIds() { manager.removeExpiredProducers(time.milliseconds() + producerIdExpirationMs + 1); } } diff --git a/kafka-merge-pr.py b/kafka-merge-pr.py index ce2e8d0674d85..a775a384356f5 100755 --- a/kafka-merge-pr.py +++ b/kafka-merge-pr.py @@ -70,7 +70,7 @@ DEV_BRANCH_NAME = "trunk" -DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "3.8.0") +DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "3.9.0") ORIGINAL_HEAD = "" diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index f0bd98776bc38..8b9c5b19eae4f 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -50,6 +50,7 @@ import org.apache.kafka.metadata.placement.StripedReplicaPlacer; import org.apache.kafka.metadata.placement.UsableBroker; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.Features; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.TimelineHashMap; @@ -407,6 +408,13 @@ public ControllerResult registerBroker( setBrokerEpoch(brokerEpoch). setRack(request.rack()). setEndPoints(listenerInfo.toBrokerRegistrationRecord()); + + if (existing != null && request.incarnationId().equals(existing.incarnationId())) { + log.info("Amending registration of broker {}", request.brokerId()); + record.setFenced(existing.fenced()); + record.setInControlledShutdown(existing.inControlledShutdown()); + } + for (BrokerRegistrationRequestData.Feature feature : request.features()) { record.features().add(processRegistrationFeature(brokerId, finalizedFeatures, feature)); } @@ -459,18 +467,19 @@ BrokerFeature processRegistrationFeature( FinalizedControllerFeatures finalizedFeatures, BrokerRegistrationRequestData.Feature feature ) { - Optional finalized = finalizedFeatures.get(feature.name()); - if (finalized.isPresent()) { - if (!VersionRange.of(feature.minSupportedVersion(), feature.maxSupportedVersion()).contains(finalized.get())) { - throw new UnsupportedVersionException("Unable to register because the broker " + - "does not support version " + finalized.get() + " of " + feature.name() + - ". It wants a version between " + feature.minSupportedVersion() + " and " + - feature.maxSupportedVersion() + ", inclusive."); - } - } else { + int defaultVersion = feature.name().equals(MetadataVersion.FEATURE_NAME) ? 1 : 0; // The default value for MetadataVersion is 1 not 0. + short finalized = finalizedFeatures.versionOrDefault(feature.name(), (short) defaultVersion); + if (!VersionRange.of(feature.minSupportedVersion(), feature.maxSupportedVersion()).contains(finalized)) { + throw new UnsupportedVersionException("Unable to register because the broker " + + "does not support version " + finalized + " of " + feature.name() + + ". It wants a version between " + feature.minSupportedVersion() + " and " + + feature.maxSupportedVersion() + ", inclusive."); + } + // A feature is not found in the finalizedFeature map if it is unknown to the controller or set to 0 (feature not enabled). + // Only log if the feature name is not known by the controller. + if (!Features.PRODUCTION_FEATURE_NAMES.contains(feature.name())) log.warn("Broker {} registered with feature {} that is unknown to the controller", brokerId, feature.name()); - } return new BrokerFeature(). setName(feature.name()). setMinSupportedVersion(feature.minSupportedVersion()). diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java index 327bbd954fc50..701c759832883 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumFeatures.java @@ -19,6 +19,7 @@ import org.apache.kafka.metadata.ControllerRegistration; import org.apache.kafka.metadata.VersionRange; +import org.apache.kafka.server.common.Features; import org.apache.kafka.server.common.MetadataVersion; import java.util.ArrayList; @@ -61,6 +62,13 @@ public static Map defaultFeatureMap(boolean enableUnstable enableUnstable ? MetadataVersion.latestTesting().featureLevel() : MetadataVersion.latestProduction().featureLevel())); + for (Features feature : Features.PRODUCTION_FEATURES) { + features.put(feature.featureName(), VersionRange.of( + 0, + enableUnstable ? + feature.latestTesting() : + feature.latestProduction())); + } return features; } diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/BrokerRegistrationTracker.java b/metadata/src/main/java/org/apache/kafka/image/publisher/BrokerRegistrationTracker.java new file mode 100644 index 0000000000000..51ac2bdfa4bd3 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/image/publisher/BrokerRegistrationTracker.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.image.publisher; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.loader.LoaderManifest; +import org.apache.kafka.metadata.BrokerRegistration; +import org.apache.kafka.server.common.MetadataVersion; +import org.slf4j.Logger; + +import java.util.List; + +/** + * Tracks the registration of a specific broker, and executes a callback if it should be refreshed. + * + * This tracker handles cases where we might want to re-register the broker. The only such case + * right now is during the transition from non-JBOD mode, to JBOD mode. In other words, the + * transition from a MetadataVersion less than 3.7-IV2, to one greater than or equal to 3.7-IV2. + * In this case, the broker registration will start out containing no directories, and we need to + * resend the BrokerRegistrationRequest to fix that. + * + * As much as possible, the goal here is to keep things simple. We just compare the desired state + * with the actual state, and try to make changes only if necessary. + */ +public class BrokerRegistrationTracker implements MetadataPublisher { + private final Logger log; + private final int id; + private final Runnable refreshRegistrationCallback; + + /** + * Create the tracker. + * + * @param id The ID of this broker. + * @param targetDirectories The directories managed by this broker. + * @param refreshRegistrationCallback Callback to run if we need to refresh the registration. + */ + public BrokerRegistrationTracker( + int id, + List targetDirectories, + Runnable refreshRegistrationCallback + ) { + this.log = new LogContext("[BrokerRegistrationTracker id=" + id + "] "). + logger(BrokerRegistrationTracker.class); + this.id = id; + this.refreshRegistrationCallback = refreshRegistrationCallback; + } + + @Override + public String name() { + return "BrokerRegistrationTracker(id=" + id + ")"; + } + + @Override + public void onMetadataUpdate( + MetadataDelta delta, + MetadataImage newImage, + LoaderManifest manifest + ) { + boolean checkBrokerRegistration = false; + if (delta.featuresDelta() != null) { + if (delta.metadataVersionChanged().isPresent()) { + if (log.isTraceEnabled()) { + log.trace("Metadata version change is present: {}", + delta.metadataVersionChanged()); + } + checkBrokerRegistration = true; + } + } + if (delta.clusterDelta() != null) { + if (delta.clusterDelta().changedBrokers().get(id) != null) { + if (log.isTraceEnabled()) { + log.trace("Broker change is present: {}", + delta.clusterDelta().changedBrokers().get(id)); + } + checkBrokerRegistration = true; + } + } + if (checkBrokerRegistration) { + if (brokerRegistrationNeedsRefresh(newImage.features().metadataVersion(), + delta.clusterDelta().broker(id))) { + refreshRegistrationCallback.run(); + } + } + } + + /** + * Check if the current broker registration needs to be refreshed. + * + * @param metadataVersion The current metadata version. + * @param registration The current broker registration, or null if there is none. + * @return True only if we should refresh. + */ + boolean brokerRegistrationNeedsRefresh( + MetadataVersion metadataVersion, + BrokerRegistration registration + ) { + // If there is no existing registration, the BrokerLifecycleManager must still be sending it. + // So we don't need to do anything yet. + if (registration == null) { + log.debug("No current broker registration to check."); + return false; + } + // Check to see if the directory list has changed. Note that this check could certainly be + // triggered spuriously. For example, if the broker's directory list has been changed in the + // past, and we are in the process of replaying that change log, we will end up here. + // That's fine because resending the broker registration does not cause any problems. And, + // of course, as soon as a snapshot is made, we will no longer need to worry about those + // old metadata log entries being replayed on startup. + if (metadataVersion.isAtLeast(MetadataVersion.IBP_3_7_IV2) && + registration.directories().isEmpty()) { + log.info("Current directory set is empty, but MV supports JBOD. Resending " + + "broker registration."); + return true; + } + log.debug("Broker registration does not need to be resent."); + return false; + } +} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/FinalizedControllerFeatures.java b/metadata/src/main/java/org/apache/kafka/metadata/FinalizedControllerFeatures.java index 05ef45d1e9667..4b311f7f72fcf 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/FinalizedControllerFeatures.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/FinalizedControllerFeatures.java @@ -40,6 +40,10 @@ public Optional get(String name) { return Optional.ofNullable(featureMap.get(name)); } + public short versionOrDefault(String name, short defaultValue) { + return featureMap.getOrDefault(name, defaultValue); + } + public Set featureNames() { return featureMap.keySet(); } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java b/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java index e48eeefbe1289..252db1a599c90 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java @@ -22,7 +22,7 @@ import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.loader.LoaderManifest; import org.apache.kafka.image.publisher.MetadataPublisher; -import org.apache.kafka.server.common.Features; +import org.apache.kafka.server.common.FinalizedFeatures; import org.slf4j.Logger; import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_KRAFT_VERSION; @@ -30,7 +30,7 @@ public class FeaturesPublisher implements MetadataPublisher { private final Logger log; - private volatile Features features = Features.fromKRaftVersion(MINIMUM_KRAFT_VERSION); + private volatile FinalizedFeatures finalizedFeatures = FinalizedFeatures.fromKRaftVersion(MINIMUM_KRAFT_VERSION); public FeaturesPublisher( LogContext logContext @@ -38,8 +38,8 @@ public FeaturesPublisher( log = logContext.logger(FeaturesPublisher.class); } - public Features features() { - return features; + public FinalizedFeatures features() { + return finalizedFeatures; } @Override @@ -54,13 +54,13 @@ public void onMetadataUpdate( LoaderManifest manifest ) { if (delta.featuresDelta() != null) { - Features newFeatures = new Features(newImage.features().metadataVersion(), + FinalizedFeatures newFinalizedFeatures = new FinalizedFeatures(newImage.features().metadataVersion(), newImage.features().finalizedVersions(), newImage.provenance().lastContainedOffset(), true); - if (!newFeatures.equals(features)) { - log.info("Loaded new metadata {}.", newFeatures); - features = newFeatures; + if (!newFinalizedFeatures.equals(finalizedFeatures)) { + log.info("Loaded new metadata {}.", newFinalizedFeatures); + finalizedFeatures = newFinalizedFeatures; } } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index 1b18c9648de4f..4498778592c6e 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -196,7 +196,7 @@ public void testConfigurationOperations() throws Throwable { ) { controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT, new BrokerRegistrationRequestData(). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_8_IV0)). + setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IVO)). setBrokerId(0). setLogDirs(Collections.singletonList(Uuid.fromString("iiaQjkRPQcuMULNII0MUeA"))). setClusterId(logEnv.clusterId())).get(); @@ -240,7 +240,7 @@ public void testDelayedConfigurationOperations() throws Throwable { ) { controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT, new BrokerRegistrationRequestData(). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_8_IV0)). + setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IVO)). setBrokerId(0). setLogDirs(Collections.singletonList(Uuid.fromString("sTbzRAMnTpahIyIPNjiLhw"))). setClusterId(logEnv.clusterId())).get(); @@ -298,7 +298,7 @@ public void testFenceMultipleBrokers() throws Throwable { new BrokerRegistrationRequestData(). setBrokerId(brokerId). setClusterId(active.clusterId()). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_7_IV0)). + setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IVO)). setIncarnationId(Uuid.randomUuid()). setListeners(listeners)); brokerEpochs.put(brokerId, reply.get().epoch()); @@ -717,7 +717,7 @@ public void testUnregisterBroker() throws Throwable { setBrokerId(0). setClusterId(active.clusterId()). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwBA")). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_8_IV0)). + setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_4_0_IVO)). setLogDirs(Collections.singletonList(Uuid.fromString("vBpaRsZVSaGsQT53wtYGtg"))). setListeners(listeners)); assertEquals(5L, reply.get().epoch()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java index 099acea2de4be..bdda58d01b6bc 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.metadata.ControllerRegistration; import org.apache.kafka.metadata.VersionRange; +import org.apache.kafka.server.common.Features; import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Test; @@ -55,6 +56,12 @@ public void testDefaultFeatureMap() { expectedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), MetadataVersion.LATEST_PRODUCTION.featureLevel())); + for (Features feature : Features.PRODUCTION_FEATURES) { + expectedFeatures.put(feature.featureName(), VersionRange.of( + 0, + feature.defaultValue(MetadataVersion.LATEST_PRODUCTION) + )); + } assertEquals(expectedFeatures, QuorumFeatures.defaultFeatureMap(false)); } @@ -64,6 +71,12 @@ public void testDefaultFeatureMapWithUnstable() { expectedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(), MetadataVersion.latestTesting().featureLevel())); + for (Features feature : Features.PRODUCTION_FEATURES) { + expectedFeatures.put(feature.featureName(), VersionRange.of( + 0, + feature.defaultValue(MetadataVersion.latestTesting()) + )); + } assertEquals(expectedFeatures, QuorumFeatures.defaultFeatureMap(true)); } diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java new file mode 100644 index 0000000000000..855a96cd8aaf3 --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.image.publisher; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.metadata.FeatureLevelRecord; +import org.apache.kafka.common.metadata.RegisterBrokerRecord; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.MetadataProvenance; +import org.apache.kafka.image.loader.LogDeltaManifest; +import org.apache.kafka.raft.LeaderAndEpoch; +import org.apache.kafka.server.common.MetadataVersion; +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.util.Arrays; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +@Timeout(value = 40) +public class BrokerRegistrationTrackerTest { + static final Uuid INCARNATION_ID = Uuid.fromString("jyjLbk31Tpa53pFrU9Y-Ng"); + + static final Uuid A = Uuid.fromString("Ahw3vXfnThqeZbb7HD1w6Q"); + + static final Uuid B = Uuid.fromString("BjOacT0OTNqIvUWIlKhahg"); + + static final Uuid C = Uuid.fromString("CVHi_iv2Rvy5_1rtPdasfg"); + + static class BrokerRegistrationTrackerTestContext { + AtomicInteger numCalls = new AtomicInteger(0); + BrokerRegistrationTracker tracker = new BrokerRegistrationTracker(1, + Arrays.asList(B, A), () -> numCalls.incrementAndGet()); + + MetadataImage image = MetadataImage.EMPTY; + + void onMetadataUpdate(MetadataDelta delta) { + MetadataProvenance provenance = new MetadataProvenance(0, 0, 0); + image = delta.apply(provenance); + LogDeltaManifest manifest = new LogDeltaManifest.Builder(). + provenance(provenance). + leaderAndEpoch(LeaderAndEpoch.UNKNOWN). + numBatches(1). + elapsedNs(1). + numBytes(1). + build(); + tracker.onMetadataUpdate(delta, image, manifest); + } + + MetadataDelta newDelta() { + return new MetadataDelta.Builder(). + setImage(image). + build(); + } + } + + @Test + public void testTrackerName() { + BrokerRegistrationTrackerTestContext ctx = new BrokerRegistrationTrackerTestContext(); + assertEquals("BrokerRegistrationTracker(id=1)", ctx.tracker.name()); + } + + @Test + public void testMetadataVersionUpdateWithoutRegistrationDoesNothing() { + BrokerRegistrationTrackerTestContext ctx = new BrokerRegistrationTrackerTestContext(); + MetadataDelta delta = ctx.newDelta(); + delta.replay(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(MetadataVersion.IBP_3_7_IV2.featureLevel())); + ctx.onMetadataUpdate(delta); + assertEquals(0, ctx.numCalls.get()); + } + + @Test + public void testBrokerUpdateWithoutNewMvDoesNothing() { + BrokerRegistrationTrackerTestContext ctx = new BrokerRegistrationTrackerTestContext(); + MetadataDelta delta = ctx.newDelta(); + delta.replay(new RegisterBrokerRecord(). + setBrokerId(1). + setIncarnationId(INCARNATION_ID). + setLogDirs(Arrays.asList(A, B, C))); + ctx.onMetadataUpdate(delta); + assertEquals(0, ctx.numCalls.get()); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testBrokerUpdateWithNewMv(boolean jbodMv) { + BrokerRegistrationTrackerTestContext ctx = new BrokerRegistrationTrackerTestContext(); + MetadataDelta delta = ctx.newDelta(); + delta.replay(new RegisterBrokerRecord(). + setBrokerId(1). + setIncarnationId(INCARNATION_ID). + setLogDirs(Arrays.asList())); + delta.replay(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(jbodMv ? MetadataVersion.IBP_3_7_IV2.featureLevel() : + MetadataVersion.IBP_3_7_IV1.featureLevel())); + ctx.onMetadataUpdate(delta); + if (jbodMv) { + assertEquals(1, ctx.numCalls.get()); + } else { + assertEquals(0, ctx.numCalls.get()); + } + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testBrokerUpdateWithNewMvWithTwoDeltas(boolean jbodMv) { + BrokerRegistrationTrackerTestContext ctx = new BrokerRegistrationTrackerTestContext(); + MetadataDelta delta = ctx.newDelta(); + delta.replay(new RegisterBrokerRecord(). + setBrokerId(1). + setIncarnationId(INCARNATION_ID). + setLogDirs(Arrays.asList())); + ctx.onMetadataUpdate(delta); + // No calls are made because MetadataVersion is 3.0-IV1 initially + assertEquals(0, ctx.numCalls.get()); + + delta = ctx.newDelta(); + delta.replay(new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(jbodMv ? MetadataVersion.IBP_3_7_IV2.featureLevel() : + MetadataVersion.IBP_3_7_IV1.featureLevel())); + ctx.onMetadataUpdate(delta); + if (jbodMv) { + assertEquals(1, ctx.numCalls.get()); + } else { + assertEquals(0, ctx.numCalls.get()); + } + } +} diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index ed09a1449e76b..10910c3db7906 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -17,6 +17,7 @@ package org.apache.kafka.raft; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; @@ -2548,6 +2549,10 @@ public OptionalLong highWatermark() { } } + public Optional voterNode(int id, String listener) { + return partitionState.lastVoterSet().voterNode(id, listener); + } + // Visible only for test QuorumState quorum() { // It's okay to return null since this method is only called by tests diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 81a6c0168ebad..522b70805047e 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -24,6 +24,7 @@ import java.util.OptionalInt; import java.util.Random; import java.util.function.Supplier; + import org.apache.kafka.common.Uuid; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java index d5a046e8f8916..16662e06ee3c4 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java @@ -28,6 +28,8 @@ import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; + +import org.apache.kafka.common.Node; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.message.VotersRecord; @@ -366,4 +368,16 @@ public static VoterSet fromInetSocketAddresses(String listener, Map voterNode(int id, String listener) { + VoterNode voterNode = voters.get(id); + if (voterNode == null) { + return Optional.empty(); + } + InetSocketAddress address = voterNode.listeners.get(listener); + if (address == null) { + return Optional.empty(); + } + return Optional.of(new Node(id, address.getHostString(), address.getPort())); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java index 14386f88f1674..22dd52ec364ba 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java @@ -26,7 +26,7 @@ final public class VoterSetHistoryTest { @Test - void testStaicVoterSet() { + void testStaticVoterSet() { VoterSet staticVoterSet = new VoterSet(VoterSetTest.voterMap(Arrays.asList(1, 2, 3), true)); VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); @@ -76,6 +76,7 @@ void testAddAt() { assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(99)); assertEquals(Optional.of(addedVoterSet), votersHistory.valueAtOrBefore(100)); + voterMap.remove(4); VoterSet removedVoterSet = new VoterSet(new HashMap<>(voterMap)); votersHistory.addAt(200, removedVoterSet); diff --git a/server-common/src/main/java/org/apache/kafka/server/common/FeatureVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/FeatureVersion.java new file mode 100644 index 0000000000000..1712d094307d6 --- /dev/null +++ b/server-common/src/main/java/org/apache/kafka/server/common/FeatureVersion.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.server.common; + +import java.util.Map; + +public interface FeatureVersion { + + /** + * The level of the feature. 0 means the feature is fully disabled, so this value should be positive. + */ + short featureLevel(); + + /** + * The name of the feature. + */ + String featureName(); + + /** + * The MetadataVersion that corresponds to this feature. Setting this MV is not required to set the feature version, + * but this MV is marked production ready if and only if this feature version is production ready. + * + * When bootstrapping, we can find the latest production feature version by finding the highest bootstrapMetadataVersion that is less than or equal + * to the given MetadataVersion ({@link MetadataVersion#LATEST_PRODUCTION} by default). Setting a feature explicitly + * will skip this mapping and allow setting the feature independently as long as it is a supported version. + * + * If feature level X is created when MetadataVersion Y is the latest production version, create a new MV Y + 1. When the feature version becomes + * production ready, set MetadataVersion Y + 1 as production ready. + * (Ie, if the current production MV is 17 when a feature version is created, create MV 18 and mark it as production ready when the feature version is production ready.) + * + * NOTE: The feature can be used without setting this metadata version. If we want to mark a dependency, do so in {@link FeatureVersion#dependencies} + */ + MetadataVersion bootstrapMetadataVersion(); + + /** + * A mapping from feature to level for all features that this feature depends on. If this feature doesn't + * depend on any others, return an empty map. + * For example, say feature X level x relies on feature Y level y: + * feature (X level x).dependencies() will return (Y -> y) + */ + Map dependencies(); +} diff --git a/server-common/src/main/java/org/apache/kafka/server/common/Features.java b/server-common/src/main/java/org/apache/kafka/server/common/Features.java index ba41104806310..eda4b85560522 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/Features.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/Features.java @@ -16,72 +16,142 @@ */ package org.apache.kafka.server.common; -import java.util.Collections; -import java.util.HashMap; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; import java.util.Map; -import java.util.Objects; +import java.util.stream.Collectors; -import static org.apache.kafka.server.common.MetadataVersion.FEATURE_NAME; +/** + * This is enum for the various features implemented for Kafka clusters. + * KIP-584: Versioning Scheme for Features introduced the idea of various features, but only added one feature -- MetadataVersion. + * KIP-1022: Formatting and Updating Features allowed for more features to be added. In order to set and update features, + * they need to be specified via the StorageTool or FeatureCommand tools. + *
      + * Having a unified enum for the features that will use a shared type in the API used to set and update them + * makes it easier to process these features. + */ +public enum Features { + + /** + * Features defined. If a feature is included in this list, and marked to be used in production they will also be specified when + * formatting a cluster via the StorageTool. MetadataVersion is handled separately, so it is not included here. + * + * See {@link TestFeatureVersion} as an example. See {@link FeatureVersion} when implementing a new feature. + */ + TEST_VERSION("test.feature.version", TestFeatureVersion.values()), + GROUP_VERSION("group.version", GroupVersion.values()); -public final class Features { - private final MetadataVersion version; - private final Map finalizedFeatures; - private final long finalizedFeaturesEpoch; + public static final Features[] FEATURES; + public static final List PRODUCTION_FEATURES; - public static Features fromKRaftVersion(MetadataVersion version) { - return new Features(version, Collections.emptyMap(), -1, true); + public static final List PRODUCTION_FEATURE_NAMES; + private final String name; + private final FeatureVersion[] featureVersions; + + Features(String name, + FeatureVersion[] featureVersions) { + this.name = name; + this.featureVersions = featureVersions; } - public Features( - MetadataVersion version, - Map finalizedFeatures, - long finalizedFeaturesEpoch, - boolean kraftMode - ) { - this.version = version; - this.finalizedFeatures = new HashMap<>(finalizedFeatures); - this.finalizedFeaturesEpoch = finalizedFeaturesEpoch; - // In KRaft mode, we always include the metadata version in the features map. - // In ZK mode, we never include it. - if (kraftMode) { - this.finalizedFeatures.put(FEATURE_NAME, version.featureLevel()); - } else { - this.finalizedFeatures.remove(FEATURE_NAME); - } + static { + Features[] enumValues = Features.values(); + FEATURES = Arrays.copyOf(enumValues, enumValues.length); + + PRODUCTION_FEATURES = Arrays.stream(FEATURES).filter(feature -> + !feature.name.equals(TEST_VERSION.featureName())).collect(Collectors.toList()); + PRODUCTION_FEATURE_NAMES = PRODUCTION_FEATURES.stream().map(feature -> + feature.name).collect(Collectors.toList()); + } + + public String featureName() { + return name; } - public MetadataVersion metadataVersion() { - return version; + public FeatureVersion[] featureVersions() { + return featureVersions; } - public Map finalizedFeatures() { - return finalizedFeatures; + public short latestProduction() { + return defaultValue(MetadataVersion.LATEST_PRODUCTION); } - public long finalizedFeaturesEpoch() { - return finalizedFeaturesEpoch; + public short latestTesting() { + return featureVersions[featureVersions.length - 1].featureLevel(); } - @Override - public boolean equals(Object o) { - if (o == null || !(o.getClass().equals(Features.class))) return false; - Features other = (Features) o; - return version == other.version && - finalizedFeatures.equals(other.finalizedFeatures) && - finalizedFeaturesEpoch == other.finalizedFeaturesEpoch; + /** + * Creates a FeatureVersion from a level. + * + * @param level the level of the feature + * @param allowUnstableFeatureVersions whether unstable versions can be used + * @return the FeatureVersionUtils.FeatureVersion for the feature the enum is based on. + * @throws IllegalArgumentException if the feature is not known. + */ + public FeatureVersion fromFeatureLevel(short level, + boolean allowUnstableFeatureVersions) { + return Arrays.stream(featureVersions).filter(featureVersion -> + featureVersion.featureLevel() == level && (allowUnstableFeatureVersions || level <= latestProduction())).findFirst().orElseThrow( + () -> new IllegalArgumentException("No feature:" + featureName() + " with feature level " + level)); } - @Override - public int hashCode() { - return Objects.hash(version, finalizedFeatures, finalizedFeaturesEpoch); + /** + * A method to validate the feature can be set. If a given feature relies on another feature, the dependencies should be + * captured in {@link FeatureVersion#dependencies()} + *

      + * For example, say feature X level x relies on feature Y level y: + * if feature X >= x then throw an error if feature Y < y. + * + * All feature levels above 0 require metadata.version=4 (IBP_3_3_IV0) in order to write the feature records to the cluster. + * + * @param feature the feature we are validating + * @param features the feature versions we have (or want to set) + * @throws IllegalArgumentException if the feature is not valid + */ + public static void validateVersion(FeatureVersion feature, Map features) { + Short metadataVersion = features.get(MetadataVersion.FEATURE_NAME); + + if (feature.featureLevel() >= 1 && (metadataVersion == null || metadataVersion < MetadataVersion.IBP_3_3_IV0.featureLevel())) + throw new IllegalArgumentException(feature.featureName() + " could not be set to " + feature.featureLevel() + + " because it depends on metadata.version=4 (" + MetadataVersion.IBP_3_3_IV0 + ")"); + + for (Map.Entry dependency: feature.dependencies().entrySet()) { + Short featureLevel = features.get(dependency.getKey()); + + if (featureLevel == null || featureLevel < dependency.getValue()) { + throw new IllegalArgumentException(feature.featureName() + " could not be set to " + feature.featureLevel() + + " because it depends on " + dependency.getKey() + " level " + dependency.getValue()); + } + } + } + + /** + * A method to return the default (latest production) level of a feature based on the metadata version provided. + * + * Every time a new feature is added, it should create a mapping from metadata version to feature version + * with {@link FeatureVersion#bootstrapMetadataVersion()}. When the feature version is production ready, the metadata + * version should be made production ready as well. + * + * @param metadataVersion the metadata version we want to use to set the default. + * @return the default version level given the feature and provided metadata version + */ + public short defaultValue(MetadataVersion metadataVersion) { + short level = 0; + for (Iterator it = Arrays.stream(featureVersions).iterator(); it.hasNext(); ) { + FeatureVersion feature = it.next(); + if (feature.bootstrapMetadataVersion().isLessThan(metadataVersion) || feature.bootstrapMetadataVersion().equals(metadataVersion)) + level = feature.featureLevel(); + else + return level; + } + return level; } - @Override - public String toString() { - return "Features" + - "(version=" + version + - ", finalizedFeatures=" + finalizedFeatures + - ", finalizedFeaturesEpoch=" + finalizedFeaturesEpoch + - ")"; + /** + * Utility method to map a list of FeatureVersion to a map of feature name to feature level + */ + public static Map featureImplsToMap(List features) { + return features.stream().collect(Collectors.toMap(FeatureVersion::featureName, FeatureVersion::featureLevel)); } } diff --git a/server-common/src/main/java/org/apache/kafka/server/common/FinalizedFeatures.java b/server-common/src/main/java/org/apache/kafka/server/common/FinalizedFeatures.java new file mode 100644 index 0000000000000..de78a3a72a883 --- /dev/null +++ b/server-common/src/main/java/org/apache/kafka/server/common/FinalizedFeatures.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.common; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +public final class FinalizedFeatures { + private final MetadataVersion metadataVersion; + private final Map finalizedFeatures; + private final long finalizedFeaturesEpoch; + + public static FinalizedFeatures fromKRaftVersion(MetadataVersion version) { + return new FinalizedFeatures(version, Collections.emptyMap(), -1, true); + } + + public FinalizedFeatures( + MetadataVersion metadataVersion, + Map finalizedFeatures, + long finalizedFeaturesEpoch, + boolean kraftMode + ) { + this.metadataVersion = metadataVersion; + this.finalizedFeatures = new HashMap<>(finalizedFeatures); + this.finalizedFeaturesEpoch = finalizedFeaturesEpoch; + // In KRaft mode, we always include the metadata version in the features map. + // In ZK mode, we never include it. + if (kraftMode) { + this.finalizedFeatures.put(MetadataVersion.FEATURE_NAME, metadataVersion.featureLevel()); + } else { + this.finalizedFeatures.remove(MetadataVersion.FEATURE_NAME); + } + } + + public MetadataVersion metadataVersion() { + return metadataVersion; + } + + public Map finalizedFeatures() { + return finalizedFeatures; + } + + public long finalizedFeaturesEpoch() { + return finalizedFeaturesEpoch; + } + + @Override + public boolean equals(Object o) { + if (o == null || !(o.getClass().equals(FinalizedFeatures.class))) return false; + FinalizedFeatures other = (FinalizedFeatures) o; + return metadataVersion == other.metadataVersion && + finalizedFeatures.equals(other.finalizedFeatures) && + finalizedFeaturesEpoch == other.finalizedFeaturesEpoch; + } + + @Override + public int hashCode() { + return Objects.hash(metadataVersion, finalizedFeatures, finalizedFeaturesEpoch); + } + + @Override + public String toString() { + return "Features" + + "(metadataVersion=" + metadataVersion + + ", finalizedFeatures=" + finalizedFeatures + + ", finalizedFeaturesEpoch=" + finalizedFeaturesEpoch + + ")"; + } +} diff --git a/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java new file mode 100644 index 0000000000000..002d7ef33f40b --- /dev/null +++ b/server-common/src/main/java/org/apache/kafka/server/common/GroupVersion.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.common; + +import java.util.Collections; +import java.util.Map; + +public enum GroupVersion implements FeatureVersion { + + // Version 1 enables the consumer rebalance protocol (KIP-848). + GV_1(1, MetadataVersion.IBP_4_0_IVO, Collections.emptyMap()); + + public static final String FEATURE_NAME = "group.version"; + + private final short featureLevel; + private final MetadataVersion bootstrapMetadataVersion; + private final Map dependencies; + + GroupVersion( + int featureLevel, + MetadataVersion bootstrapMetadataVersion, + Map dependencies + ) { + this.featureLevel = (short) featureLevel; + this.bootstrapMetadataVersion = bootstrapMetadataVersion; + this.dependencies = dependencies; + } + + @Override + public short featureLevel() { + return featureLevel; + } + + @Override + public String featureName() { + return FEATURE_NAME; + } + + @Override + public MetadataVersion bootstrapMetadataVersion() { + return bootstrapMetadataVersion; + } + + @Override + public Map dependencies() { + return dependencies; + } +} diff --git a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java index 22e545c11292c..26b67321e4a06 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java @@ -202,7 +202,10 @@ public enum MetadataVersion { IBP_3_7_IV4(19, "3.7", "IV4", false), // Add ELR related supports (KIP-966). - IBP_3_8_IV0(20, "3.8", "IV0", true); + IBP_3_8_IV0(20, "3.8", "IV0", true), + + // Introduce version 1 of the GroupVersion feature (KIP-848). + IBP_4_0_IVO(21, "4.0", "IV0", false); // NOTES when adding a new version: // Update the default version in @ClusterTest annotation to point to the latest version @@ -258,6 +261,10 @@ public enum MetadataVersion { this.didMetadataChange = didMetadataChange; } + public String featureName() { + return FEATURE_NAME; + } + public short featureLevel() { return featureLevel; } diff --git a/server-common/src/main/java/org/apache/kafka/server/common/TestFeatureVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/TestFeatureVersion.java new file mode 100644 index 0000000000000..2387f838341c4 --- /dev/null +++ b/server-common/src/main/java/org/apache/kafka/server/common/TestFeatureVersion.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.common; + +import java.util.Collections; +import java.util.Map; + +public enum TestFeatureVersion implements FeatureVersion { + + // TEST_1 released right before MV 3.7-IVO was released, and it has no dependencies + TEST_1(1, MetadataVersion.IBP_3_7_IV0, Collections.emptyMap()), + // TEST_2 released right before MV 3.8-IVO was released, and it depends on this metadata version + TEST_2(2, MetadataVersion.IBP_3_8_IV0, Collections.singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_8_IV0.featureLevel())); + + private final short featureLevel; + private final MetadataVersion metadataVersionMapping; + private final Map dependencies; + + public static final String FEATURE_NAME = "test.feature.version"; + + TestFeatureVersion(int featureLevel, MetadataVersion metadataVersionMapping, Map dependencies) { + this.featureLevel = (short) featureLevel; + this.metadataVersionMapping = metadataVersionMapping; + this.dependencies = dependencies; + } + + public short featureLevel() { + return featureLevel; + } + + public String featureName() { + return FEATURE_NAME; + } + + public MetadataVersion bootstrapMetadataVersion() { + return metadataVersionMapping; + } + + public Map dependencies() { + return dependencies; + } +} diff --git a/server-common/src/main/java/org/apache/kafka/server/config/QuotaConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/QuotaConfigs.java index 88b49b82827e4..1005672a8d6de 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/QuotaConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/QuotaConfigs.java @@ -121,6 +121,20 @@ private static void buildUserClientQuotaConfigDef(ConfigDef configDef) { ConfigDef.Importance.MEDIUM, CONTROLLER_MUTATION_RATE_DOC); } + public static ConfigDef brokerQuotaConfigs() { + return new ConfigDef() + // Round minimum value down, to make it easier for users. + .define(QuotaConfigs.LEADER_REPLICATION_THROTTLED_RATE_CONFIG, ConfigDef.Type.LONG, + QuotaConfigs.QUOTA_BYTES_PER_SECOND_DEFAULT, ConfigDef.Range.atLeast(0), + ConfigDef.Importance.MEDIUM, QuotaConfigs.LEADER_REPLICATION_THROTTLED_RATE_DOC) + .define(QuotaConfigs.FOLLOWER_REPLICATION_THROTTLED_RATE_CONFIG, ConfigDef.Type.LONG, + QuotaConfigs.QUOTA_BYTES_PER_SECOND_DEFAULT, ConfigDef.Range.atLeast(0), + ConfigDef.Importance.MEDIUM, QuotaConfigs.FOLLOWER_REPLICATION_THROTTLED_RATE_DOC) + .define(QuotaConfigs.REPLICA_ALTER_LOG_DIRS_IO_MAX_BYTES_PER_SECOND_CONFIG, ConfigDef.Type.LONG, + QuotaConfigs.QUOTA_BYTES_PER_SECOND_DEFAULT, ConfigDef.Range.atLeast(0), + ConfigDef.Importance.MEDIUM, QuotaConfigs.REPLICA_ALTER_LOG_DIRS_IO_MAX_BYTES_PER_SECOND_DOC); + } + public static ConfigDef userAndClientQuotaConfigs() { ConfigDef configDef = new ConfigDef(); buildUserClientQuotaConfigDef(configDef); diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 8b4fed8375f6c..17d4e683978b1 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -190,4 +190,9 @@ public class ServerLogConfigs { public static final long LOG_INITIAL_TASK_DELAY_MS_DEFAULT = 30 * 1000L; public static final String LOG_INITIAL_TASK_DELAY_MS_DOC = "The initial task delay in millisecond when initializing " + "tasks in LogManager. This should be used for testing only."; + + public static final String LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG = LOG_PREFIX + "dir.failure.timeout.ms"; + public static final Long LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT = 30000L; + public static final String LOG_DIR_FAILURE_TIMEOUT_MS_DOC = "If the broker is unable to successfully communicate to the controller that some log " + + "directory has failed for longer than this time, the broker will fail and shut down."; } diff --git a/server-common/src/main/java/org/apache/kafka/server/util/Csv.java b/server-common/src/main/java/org/apache/kafka/server/util/Csv.java index f164dd1141460..61f143bdc3f19 100644 --- a/server-common/src/main/java/org/apache/kafka/server/util/Csv.java +++ b/server-common/src/main/java/org/apache/kafka/server/util/Csv.java @@ -16,8 +16,12 @@ */ package org.apache.kafka.server.util; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; public class Csv { @@ -38,4 +42,16 @@ public static Map parseCsvMap(String str) { } return map; } + + /** + * Parse a comma separated string into a sequence of strings. + * Whitespace surrounding the comma will be removed. + */ + public static List parseCsvList(String csvList) { + if (csvList == null || csvList.isEmpty()) { + return Collections.emptyList(); + } else { + return Stream.of(csvList.split("\\s*,\\s*")).filter(v -> !v.isEmpty()).collect(Collectors.toList()); + } + } } diff --git a/server-common/src/test/java/org/apache/kafka/server/common/FeaturesTest.java b/server-common/src/test/java/org/apache/kafka/server/common/FeaturesTest.java index c3d8e0f03198a..e11e554002e72 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/FeaturesTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/FeaturesTest.java @@ -14,37 +14,116 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.server.common; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; -import static org.apache.kafka.server.common.MetadataVersion.FEATURE_NAME; -import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_KRAFT_VERSION; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class FeaturesTest { + + @ParameterizedTest + @EnumSource(Features.class) + public void testFromFeatureLevelAllFeatures(Features feature) { + FeatureVersion[] featureImplementations = feature.featureVersions(); + int numFeatures = featureImplementations.length; + short latestProductionLevel = feature.latestProduction(); + + for (short i = 1; i < numFeatures; i++) { + short level = i; + if (latestProductionLevel < i) { + assertEquals(featureImplementations[i - 1], feature.fromFeatureLevel(level, true)); + assertThrows(IllegalArgumentException.class, () -> feature.fromFeatureLevel(level, false)); + } else { + assertEquals(featureImplementations[i - 1], feature.fromFeatureLevel(level, false)); + } + } + } + + @ParameterizedTest + @EnumSource(Features.class) + public void testValidateVersionAllFeatures(Features feature) { + for (FeatureVersion featureImpl : feature.featureVersions()) { + // Ensure the minimum bootstrap metadata version is included if no metadata version dependency. + Map deps = new HashMap<>(); + deps.putAll(featureImpl.dependencies()); + if (!deps.containsKey(MetadataVersion.FEATURE_NAME)) { + deps.put(MetadataVersion.FEATURE_NAME, MetadataVersion.MINIMUM_BOOTSTRAP_VERSION.featureLevel()); + } + + // Ensure that the feature is valid given the typical metadataVersionMapping and the dependencies. + // Note: Other metadata versions are valid, but this one should always be valid. + Features.validateVersion(featureImpl, deps); + } + } -class FeaturesTest { @Test - public void testKRaftModeFeatures() { - Features features = new Features(MINIMUM_KRAFT_VERSION, - Collections.singletonMap("foo", (short) 2), 123, true); - assertEquals(MINIMUM_KRAFT_VERSION.featureLevel(), - features.finalizedFeatures().get(FEATURE_NAME)); - assertEquals((short) 2, - features.finalizedFeatures().get("foo")); - assertEquals(2, features.finalizedFeatures().size()); + public void testInvalidValidateVersion() { + // No MetadataVersion is invalid + assertThrows(IllegalArgumentException.class, + () -> Features.validateVersion( + TestFeatureVersion.TEST_1, + Collections.emptyMap() + ) + ); + + // Using too low of a MetadataVersion is invalid + assertThrows(IllegalArgumentException.class, + () -> Features.validateVersion( + TestFeatureVersion.TEST_1, + Collections.singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_2_8_IV0.featureLevel()) + ) + ); + + // Using a version that is lower than the dependency will fail. + assertThrows(IllegalArgumentException.class, + () -> Features.validateVersion( + TestFeatureVersion.TEST_2, + Collections.singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_7_IV0.featureLevel()) + ) + ); + } + + @ParameterizedTest + @EnumSource(Features.class) + public void testDefaultValueAllFeatures(Features feature) { + for (FeatureVersion featureImpl : feature.featureVersions()) { + assertEquals(feature.defaultValue(featureImpl.bootstrapMetadataVersion()), featureImpl.featureLevel(), + "Failed to get the correct default for " + featureImpl); + } + } + + @ParameterizedTest + @EnumSource(Features.class) + public void testLatestProductionMapsToLatestMetadataVersion(Features features) { + assertEquals(features.latestProduction(), features.defaultValue(MetadataVersion.LATEST_PRODUCTION)); + } + + @ParameterizedTest + @EnumSource(MetadataVersion.class) + public void testDefaultTestVersion(MetadataVersion metadataVersion) { + short expectedVersion; + if (!metadataVersion.isLessThan(MetadataVersion.IBP_3_8_IV0)) { + expectedVersion = 2; + } else if (!metadataVersion.isLessThan(MetadataVersion.IBP_3_7_IV0)) { + expectedVersion = 1; + } else { + expectedVersion = 0; + } + assertEquals(expectedVersion, Features.TEST_VERSION.defaultValue(metadataVersion)); } @Test - public void testZkModeFeatures() { - Features features = new Features(MINIMUM_KRAFT_VERSION, - Collections.singletonMap("foo", (short) 2), 123, false); - assertNull(features.finalizedFeatures().get(FEATURE_NAME)); - assertEquals((short) 2, - features.finalizedFeatures().get("foo")); - assertEquals(1, features.finalizedFeatures().size()); + public void testUnstableTestVersion() { + assertThrows(IllegalArgumentException.class, () -> + Features.TEST_VERSION.fromFeatureLevel(Features.TEST_VERSION.latestTesting(), false)); + Features.TEST_VERSION.fromFeatureLevel(Features.TEST_VERSION.latestTesting(), true); } } diff --git a/server-common/src/test/java/org/apache/kafka/server/common/FinalizedFeaturesTest.java b/server-common/src/test/java/org/apache/kafka/server/common/FinalizedFeaturesTest.java new file mode 100644 index 0000000000000..ae6ca998df2da --- /dev/null +++ b/server-common/src/test/java/org/apache/kafka/server/common/FinalizedFeaturesTest.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.server.common; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; + +import static org.apache.kafka.server.common.MetadataVersion.FEATURE_NAME; +import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_KRAFT_VERSION; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +class FinalizedFeaturesTest { + @Test + public void testKRaftModeFeatures() { + FinalizedFeatures finalizedFeatures = new FinalizedFeatures(MINIMUM_KRAFT_VERSION, + Collections.singletonMap("foo", (short) 2), 123, true); + assertEquals(MINIMUM_KRAFT_VERSION.featureLevel(), + finalizedFeatures.finalizedFeatures().get(FEATURE_NAME)); + assertEquals((short) 2, + finalizedFeatures.finalizedFeatures().get("foo")); + assertEquals(2, finalizedFeatures.finalizedFeatures().size()); + } + + @Test + public void testZkModeFeatures() { + FinalizedFeatures finalizedFeatures = new FinalizedFeatures(MINIMUM_KRAFT_VERSION, + Collections.singletonMap("foo", (short) 2), 123, false); + assertNull(finalizedFeatures.finalizedFeatures().get(FEATURE_NAME)); + assertEquals((short) 2, + finalizedFeatures.finalizedFeatures().get("foo")); + assertEquals(1, finalizedFeatures.finalizedFeatures().size()); + } +} diff --git a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java index bd9f2594b0080..cfcdcf3afe66c 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java @@ -47,6 +47,7 @@ public void testKRaftFeatureLevelsAtAndAfter3_0_IV1() { } @Test + @SuppressWarnings("checkstyle:JavaNCSS") public void testFromVersionString() { assertEquals(IBP_0_8_0, MetadataVersion.fromVersionString("0.8.0")); assertEquals(IBP_0_8_0, MetadataVersion.fromVersionString("0.8.0.0")); @@ -184,6 +185,8 @@ public void testFromVersionString() { assertEquals(IBP_3_7_IV4, MetadataVersion.fromVersionString("3.7-IV4")); assertEquals(IBP_3_8_IV0, MetadataVersion.fromVersionString("3.8-IV0")); + + assertEquals(IBP_4_0_IVO, MetadataVersion.fromVersionString("4.0-IV0")); } @Test @@ -243,6 +246,8 @@ public void testShortVersion() { assertEquals("3.7", IBP_3_7_IV2.shortVersion()); assertEquals("3.7", IBP_3_7_IV3.shortVersion()); assertEquals("3.7", IBP_3_7_IV4.shortVersion()); + assertEquals("3.8", IBP_3_8_IV0.shortVersion()); + assertEquals("4.0", IBP_4_0_IVO.shortVersion()); } @Test @@ -292,6 +297,7 @@ public void testVersion() { assertEquals("3.7-IV3", IBP_3_7_IV3.version()); assertEquals("3.7-IV4", IBP_3_7_IV4.version()); assertEquals("3.8-IV0", IBP_3_8_IV0.version()); + assertEquals("4.0-IV0", IBP_4_0_IVO.version()); } @Test diff --git a/server-common/src/test/java/org/apache/kafka/server/util/CsvTest.java b/server-common/src/test/java/org/apache/kafka/server/util/CsvTest.java index 46d54b42c2bb7..65334a607b231 100644 --- a/server-common/src/test/java/org/apache/kafka/server/util/CsvTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/util/CsvTest.java @@ -18,21 +18,19 @@ import org.junit.jupiter.api.Test; +import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; public class CsvTest { @Test public void testCsvMap() { - String emptyString = ""; - Map emptyMap = Csv.parseCsvMap(emptyString); - Map emptyStringMap = Collections.emptyMap(); - assertNotNull(emptyMap); - assertEquals(emptyStringMap, emptyStringMap); + Map emptyMap = Csv.parseCsvMap(""); + assertEquals(Collections.emptyMap(), emptyMap); String kvPairsIpV6 = "a:b:c:v,a:b:c:v"; Map ipv6Map = Csv.parseCsvMap(kvPairsIpV6); @@ -60,4 +58,16 @@ public void testCsvMap() { assertEquals("value", entry.getValue()); } } + + @Test + public void testCsvList() { + List emptyList = Csv.parseCsvList(""); + assertEquals(Collections.emptyList(), emptyList); + + List emptyListFromNullString = Csv.parseCsvList(null); + assertEquals(Collections.emptyList(), emptyListFromNullString); + + List csvList = Csv.parseCsvList("a,b ,c, d,,e,"); + assertEquals(Arrays.asList("a", "b", "c", "d", "e"), csvList); + } } diff --git a/server/src/main/java/org/apache/kafka/server/config/Defaults.java b/server/src/main/java/org/apache/kafka/server/config/Defaults.java deleted file mode 100644 index ce9aeab4a2456..0000000000000 --- a/server/src/main/java/org/apache/kafka/server/config/Defaults.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.server.config; - -import org.apache.kafka.clients.CommonClientConfigs; -public class Defaults { - /** ********* General Configuration *********/ - public static final boolean BROKER_ID_GENERATION_ENABLE = true; - public static final int MAX_RESERVED_BROKER_ID = 1000; - public static final int BROKER_ID = -1; - public static final int NUM_NETWORK_THREADS = 3; - public static final int NUM_IO_THREADS = 8; - public static final int BACKGROUND_THREADS = 10; - public static final int QUEUED_MAX_REQUESTS = 500; - public static final int QUEUED_MAX_REQUEST_BYTES = -1; - public static final boolean DELETE_TOPIC_ENABLE = true; - public static final int REQUEST_TIMEOUT_MS = 30000; - public static final long CONNECTION_SETUP_TIMEOUT_MS = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS; - public static final long CONNECTION_SETUP_TIMEOUT_MAX_MS = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS; - - /** ********* KRaft mode configs *********/ - public static final int EMPTY_NODE_ID = -1; - public static final long SERVER_MAX_STARTUP_TIME_MS = Long.MAX_VALUE; - public static final int MIGRATION_METADATA_MIN_BATCH_SIZE = 200; - - /** ********* Authorizer Configuration *********/ - public static final String AUTHORIZER_CLASS_NAME = ""; - - /** ********* Controlled shutdown configuration *********/ - public static final int CONTROLLED_SHUTDOWN_MAX_RETRIES = 3; - public static final int CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS = 5000; - public static final boolean CONTROLLED_SHUTDOWN_ENABLE = true; - - /** ********* Fetch Configuration *********/ - public static final int MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS = 1000; - public static final int FETCH_MAX_BYTES = 55 * 1024 * 1024; - - /** ********* Request Limit Configuration ***********/ - public static final int MAX_REQUEST_PARTITION_SIZE_LIMIT = 2000; - - - /** ********* Delegation Token Configuration *********/ - public static final long DELEGATION_TOKEN_MAX_LIFE_TIME_MS = 7 * 24 * 60 * 60 * 1000L; - public static final long DELEGATION_TOKEN_EXPIRY_TIME_MS = 24 * 60 * 60 * 1000L; - public static final long DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS = 1 * 60 * 60 * 1000L; -} diff --git a/server/src/main/java/org/apache/kafka/server/config/DelegationTokenManagerConfigs.java b/server/src/main/java/org/apache/kafka/server/config/DelegationTokenManagerConfigs.java new file mode 100644 index 0000000000000..bf5704ed89f24 --- /dev/null +++ b/server/src/main/java/org/apache/kafka/server/config/DelegationTokenManagerConfigs.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.config; + +public class DelegationTokenManagerConfigs { + /** ********* Delegation Token Configuration ****************/ + public static final String DELEGATION_TOKEN_SECRET_KEY_CONFIG = "delegation.token.secret.key"; + public static final String DELEGATION_TOKEN_SECRET_KEY_DOC = "Secret key to generate and verify delegation tokens. The same key must be configured across all the brokers. " + + " If using Kafka with KRaft, the key must also be set across all controllers. " + + " If the key is not set or set to empty string, brokers will disable the delegation token support."; + + @Deprecated + public static final String DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG = "delegation.token.master.key"; + public static final String DELEGATION_TOKEN_SECRET_KEY_ALIAS_DOC = "DEPRECATED: An alias for " + DELEGATION_TOKEN_SECRET_KEY_CONFIG + ", which should be used instead of this config."; + + public static final String DELEGATION_TOKEN_MAX_LIFETIME_CONFIG = "delegation.token.max.lifetime.ms"; + public static final long DELEGATION_TOKEN_MAX_LIFE_TIME_MS_DEFAULT = 7 * 24 * 60 * 60 * 1000L; + public static final String DELEGATION_TOKEN_MAX_LIFE_TIME_DOC = "The token has a maximum lifetime beyond which it cannot be renewed anymore. Default value 7 days."; + + public static final String DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG = "delegation.token.expiry.time.ms"; + public static final long DELEGATION_TOKEN_EXPIRY_TIME_MS_DEFAULT = 24 * 60 * 60 * 1000L; + public static final String DELEGATION_TOKEN_EXPIRY_TIME_MS_DOC = "The token validity time in milliseconds before the token needs to be renewed. Default value 1 day."; + + public static final String DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG = "delegation.token.expiry.check.interval.ms"; + public static final long DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_DEFAULT = 60 * 60 * 1000L; + public static final String DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_DOC = "Scan interval to remove expired delegation tokens."; +} diff --git a/server/src/main/java/org/apache/kafka/server/config/KafkaSecurityConfigs.java b/server/src/main/java/org/apache/kafka/server/config/KafkaSecurityConfigs.java deleted file mode 100644 index 271ed16887084..0000000000000 --- a/server/src/main/java/org/apache/kafka/server/config/KafkaSecurityConfigs.java +++ /dev/null @@ -1,239 +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.server.config; - -import org.apache.kafka.common.config.SaslConfigs; -import org.apache.kafka.common.config.SecurityConfig; -import org.apache.kafka.common.config.SslClientAuth; -import org.apache.kafka.common.config.SslConfigs; -import org.apache.kafka.common.config.internals.BrokerSecurityConfigs; -import org.apache.kafka.common.security.auth.KafkaPrincipalBuilder; -import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder; - -import java.util.List; -import java.util.Locale; - -/** - * Common home for broker-side security configs which need to be accessible from the libraries shared - * between the broker and the multiple modules in Kafka. - * - * Note this is an internal API and subject to change without notice. - */ -public class KafkaSecurityConfigs { - - /** ********* SSL Configuration ****************/ - public final static String SSL_PROTOCOL_CONFIG = SslConfigs.SSL_PROTOCOL_CONFIG; - public final static String SSL_PROTOCOL_DOC = SslConfigs.SSL_PROTOCOL_DOC; - public static final String SSL_PROTOCOL_DEFAULT = SslConfigs.DEFAULT_SSL_PROTOCOL; - - public final static String SSL_PROVIDER_CONFIG = SslConfigs.SSL_PROVIDER_CONFIG; - public final static String SSL_PROVIDER_DOC = SslConfigs.SSL_PROVIDER_DOC; - - public final static String SSL_CIPHER_SUITES_CONFIG = SslConfigs.SSL_CIPHER_SUITES_CONFIG; - public final static String SSL_CIPHER_SUITES_DOC = SslConfigs.SSL_CIPHER_SUITES_DOC; - - public final static String SSL_ENABLED_PROTOCOLS_CONFIG = SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG; - public final static String SSL_ENABLED_PROTOCOLS_DOC = SslConfigs.SSL_ENABLED_PROTOCOLS_DOC; - public static final String SSL_ENABLED_PROTOCOLS_DEFAULTS = SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS; - - public final static String SSL_KEYSTORE_TYPE_CONFIG = SslConfigs.SSL_KEYSTORE_TYPE_CONFIG; - public final static String SSL_KEYSTORE_TYPE_DOC = SslConfigs.SSL_KEYSTORE_TYPE_DOC; - public static final String SSL_KEYSTORE_TYPE_DEFAULT = SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE; - - public final static String SSL_KEYSTORE_LOCATION_CONFIG = SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG; - public final static String SSL_KEYSTORE_LOCATION_DOC = SslConfigs.SSL_KEYSTORE_LOCATION_DOC; - - public final static String SSL_KEYSTORE_PASSWORD_CONFIG = SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG; - public final static String SSL_KEYSTORE_PASSWORD_DOC = SslConfigs.SSL_KEYSTORE_PASSWORD_DOC; - - public final static String SSL_KEY_PASSWORD_CONFIG = SslConfigs.SSL_KEY_PASSWORD_CONFIG; - public final static String SSL_KEY_PASSWORD_DOC = SslConfigs.SSL_KEY_PASSWORD_DOC; - - public final static String SSL_KEYSTORE_KEY_CONFIG = SslConfigs.SSL_KEYSTORE_KEY_CONFIG; - public final static String SSL_KEYSTORE_KEY_DOC = SslConfigs.SSL_KEYSTORE_KEY_DOC; - - public final static String SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG = SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG; - public final static String SSL_KEYSTORE_CERTIFICATE_CHAIN_DOC = SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_DOC; - public final static String SSL_TRUSTSTORE_TYPE_CONFIG = SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG; - public final static String SSL_TRUSTSTORE_TYPE_DOC = SslConfigs.SSL_TRUSTSTORE_TYPE_DOC; - public static final String SSL_TRUSTSTORE_TYPE_DEFAULT = SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE; - - public final static String SSL_TRUSTSTORE_LOCATION_CONFIG = SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG; - public final static String SSL_TRUSTSTORE_PASSWORD_DOC = SslConfigs.SSL_TRUSTSTORE_PASSWORD_DOC; - - public final static String SSL_TRUSTSTORE_PASSWORD_CONFIG = SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG; - public final static String SSL_TRUSTSTORE_LOCATION_DOC = SslConfigs.SSL_TRUSTSTORE_LOCATION_DOC; - - public final static String SSL_TRUSTSTORE_CERTIFICATES_CONFIG = SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG; - public final static String SSL_TRUSTSTORE_CERTIFICATES_DOC = SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_DOC; - - public final static String SSL_KEYMANAGER_ALGORITHM_CONFIG = SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG; - public final static String SSL_KEYMANAGER_ALGORITHM_DOC = SslConfigs.SSL_KEYMANAGER_ALGORITHM_DOC; - public static final String SSL_KEYMANAGER_ALGORITHM_DEFAULT = SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM; - - public final static String SSL_TRUSTMANAGER_ALGORITHM_CONFIG = SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG; - public final static String SSL_TRUSTMANAGER_ALGORITHM_DOC = SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC; - public static final String SSL_TRUSTMANAGER_ALGORITHM_DEFAULT = SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM; - - public final static String SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG = SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG; - public final static String SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC = SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC; - public static final String SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DEFAULT = SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM; - - public final static String SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG = SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG; - public final static String SSL_SECURE_RANDOM_IMPLEMENTATION_DOC = SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_DOC; - - public final static String SSL_CLIENT_AUTH_CONFIG = BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG; - public final static String SSL_CLIENT_AUTH_DOC = BrokerSecurityConfigs.SSL_CLIENT_AUTH_DOC; - public static final String SSL_CLIENT_AUTH_DEFAULT = SslClientAuth.NONE.name().toLowerCase(Locale.ROOT); - public static final String[] SSL_CLIENT_AUTHENTICATION_VALID_VALUES = SslClientAuth.VALUES.stream() - .map(v -> v.toString().toLowerCase(Locale.ROOT)).toArray(String[]::new); - - public final static String SSL_PRINCIPAL_MAPPING_RULES_CONFIG = BrokerSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_CONFIG; - public final static String SSL_PRINCIPAL_MAPPING_RULES_DOC = BrokerSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_DOC; - public static final String SSL_PRINCIPAL_MAPPING_RULES_DEFAULT = BrokerSecurityConfigs.DEFAULT_SSL_PRINCIPAL_MAPPING_RULES; - - public final static String SSL_ENGINE_FACTORY_CLASS_CONFIG = SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG; - public final static String SSL_ENGINE_FACTORY_CLASS_DOC = SslConfigs.SSL_ENGINE_FACTORY_CLASS_DOC; - - public final static String SSL_ALLOW_DN_CHANGES_CONFIG = BrokerSecurityConfigs.SSL_ALLOW_DN_CHANGES_CONFIG; - public final static String SSL_ALLOW_DN_CHANGES_DOC = BrokerSecurityConfigs.SSL_ALLOW_DN_CHANGES_DOC; - public final static boolean SSL_ALLOW_DN_CHANGES_DEFAULT = BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_DN_CHANGES_VALUE; - - public final static String SSL_ALLOW_SAN_CHANGES_CONFIG = BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_CONFIG; - public final static String SSL_ALLOW_SAN_CHANGES_DOC = BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_DOC; - public final static boolean SSL_ALLOW_SAN_CHANGES_DEFAULT = BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE; - - /** ********* SASL Configuration ****************/ - public final static String SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG = BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG; - public final static String SASL_MECHANISM_INTER_BROKER_PROTOCOL_DOC = BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_DOC; - - public static final String SASL_MECHANISM_INTER_BROKER_PROTOCOL_DEFAULT = SaslConfigs.DEFAULT_SASL_MECHANISM; - - public final static String SASL_JAAS_CONFIG = SaslConfigs.SASL_JAAS_CONFIG; - public final static String SASL_JAAS_CONFIG_DOC = SaslConfigs.SASL_JAAS_CONFIG_DOC; - public final static String SASL_ENABLED_MECHANISMS_CONFIG = BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG; - public final static String SASL_ENABLED_MECHANISMS_DOC = BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_DOC; - public static final List SASL_ENABLED_MECHANISMS_DEFAULT = BrokerSecurityConfigs.DEFAULT_SASL_ENABLED_MECHANISMS; - - public final static String SASL_SERVER_CALLBACK_HANDLER_CLASS_CONFIG = BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS; - public final static String SASL_SERVER_CALLBACK_HANDLER_CLASS_DOC = BrokerSecurityConfigs.SASL_SERVER_CALLBACK_HANDLER_CLASS_DOC; - public final static String SASL_CLIENT_CALLBACK_HANDLER_CLASS_CONFIG = SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS; - public final static String SASL_CLIENT_CALLBACK_HANDLER_CLASS_DOC = SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_DOC; - public final static String SASL_LOGIN_CLASS_CONFIG = SaslConfigs.SASL_LOGIN_CLASS; - public final static String SASL_LOGIN_CLASS_DOC = SaslConfigs.SASL_LOGIN_CLASS_DOC; - public final static String SASL_LOGIN_CALLBACK_HANDLER_CLASS_CONFIG = SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS; - public final static String SASL_LOGIN_CALLBACK_HANDLER_CLASS_DOC = SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS_DOC; - public final static String SASL_KERBEROS_SERVICE_NAME_CONFIG = SaslConfigs.SASL_KERBEROS_SERVICE_NAME; - public final static String SASL_KERBEROS_SERVICE_NAME_DOC = SaslConfigs.SASL_KERBEROS_SERVICE_NAME_DOC; - public final static String SASL_KERBEROS_KINIT_CMD_CONFIG = SaslConfigs.SASL_KERBEROS_KINIT_CMD; - public final static String SASL_KERBEROS_KINIT_CMD_DOC = SaslConfigs.SASL_KERBEROS_KINIT_CMD_DOC; - public static final String SASL_KERBEROS_KINIT_CMD_DEFAULT = SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD; - - public final static String SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_CONFIG = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR; - public final static String SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC; - public static final double SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DEFAULT = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR; - - public final static String SASL_KERBEROS_TICKET_RENEW_JITTER_CONFIG = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER; - public final static String SASL_KERBEROS_TICKET_RENEW_JITTER_DOC = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DOC; - public static final double SASL_KERBEROS_TICKET_RENEW_JITTER_DEFAULT = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER; - - public final static String SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_CONFIG = SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN; - public final static String SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC = SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC; - public static final long SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DEFAULT = SaslConfigs.DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN; - - public final static String SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG = BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_CONFIG; - public final static String SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DOC = BrokerSecurityConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DOC; - public static final List SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES_DEFAULT = BrokerSecurityConfigs.DEFAULT_SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES; - - public final static String SASL_LOGIN_REFRESH_WINDOW_FACTOR_CONFIG = SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR; - public final static String SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC = SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_FACTOR_DOC; - public static final double SASL_LOGIN_REFRESH_WINDOW_FACTOR_DEFAULT = SaslConfigs.DEFAULT_LOGIN_REFRESH_WINDOW_FACTOR; - - public final static String SASL_LOGIN_REFRESH_WINDOW_JITTER_CONFIG = SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER; - public final static String SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC = SaslConfigs.SASL_LOGIN_REFRESH_WINDOW_JITTER_DOC; - public static final double SASL_LOGIN_REFRESH_WINDOW_JITTER_DEFAULT = SaslConfigs.DEFAULT_LOGIN_REFRESH_WINDOW_JITTER; - - public final static String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_CONFIG = SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS; - public final static String SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC = SaslConfigs.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DOC; - public static final short SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS_DEFAULT = SaslConfigs.DEFAULT_LOGIN_REFRESH_MIN_PERIOD_SECONDS; - - public final static String SASL_LOGIN_REFRESH_BUFFER_SECONDS_CONFIG = SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS; - public final static String SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC = SaslConfigs.SASL_LOGIN_REFRESH_BUFFER_SECONDS_DOC; - public static final short SASL_LOGIN_REFRESH_BUFFER_SECONDS_DEFAULT = SaslConfigs.DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS; - - public final static String SASL_LOGIN_CONNECT_TIMEOUT_MS_CONFIG = SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS; - public final static String SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC = SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS_DOC; - public final static String SASL_LOGIN_READ_TIMEOUT_MS_CONFIG = SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS; - public final static String SASL_LOGIN_READ_TIMEOUT_MS_DOC = SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS_DOC; - public final static String SASL_LOGIN_RETRY_BACKOFF_MAX_MS_CONFIG = SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS; - public final static String SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC = SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DOC; - public static final long SASL_LOGIN_RETRY_BACKOFF_MAX_MS_DEFAULT = SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MAX_MS; - - public final static String SASL_LOGIN_RETRY_BACKOFF_MS_CONFIG = SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS; - public final static String SASL_LOGIN_RETRY_BACKOFF_MS_DOC = SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MS_DOC; - public static final long SASL_LOGIN_RETRY_BACKOFF_MS_DEFAULT = SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MS; - - public final static String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_CONFIG = SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME; - public final static String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC = SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DOC; - public static final String SASL_OAUTHBEARER_SCOPE_CLAIM_NAME_DEFAULT = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME; - - public final static String SASL_OAUTHBEARER_SUB_CLAIM_NAME_CONFIG = SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME; - public final static String SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC = SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME_DOC; - public static final String SASL_OAUTHBEARER_SUB_CLAIM_NAME_DEFAULT = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME; - - public final static String SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_CONFIG = SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; - public final static String SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC = SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL_DOC; - - public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_CONFIG = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL; - public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_URL_DOC; - public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_CONFIG = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS; - public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DOC; - public static final long SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS_DEFAULT = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS; - public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_CONFIG = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS; - public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC; - public static final long SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DEFAULT = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS; - - public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_CONFIG = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS; - public final static String SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC = SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC; - public static final long SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DEFAULT = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS; - - public final static String SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_CONFIG = SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS; - public final static String SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC = SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC; - public static final int SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DEFAULT = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS; - - public final static String SASL_OAUTHBEARER_EXPECTED_AUDIENCE_CONFIG = SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE; - public final static String SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC = SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC; - public final static String SASL_OAUTHBEARER_EXPECTED_ISSUER_CONFIG = SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER; - public final static String SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC = SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC; - - /** ******** Common Security Configuration *************/ - public final static String PRINCIPAL_BUILDER_CLASS_CONFIG = BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG; - public final static String PRINCIPAL_BUILDER_CLASS_DOC = BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_DOC; - public static final Class PRINCIPAL_BUILDER_CLASS_DEFAULT = DefaultKafkaPrincipalBuilder.class; - - public final static String CONNECTIONS_MAX_REAUTH_MS_CONFIG = BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS; - public final static String CONNECTIONS_MAX_REAUTH_MS_DOC = BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_DOC; - public static final long CONNECTIONS_MAX_REAUTH_MS_DEFAULT = BrokerSecurityConfigs.DEFAULT_CONNECTIONS_MAX_REAUTH_MS; - - public final static String SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG = BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_CONFIG; - public final static String SASL_SERVER_MAX_RECEIVE_SIZE_DOC = BrokerSecurityConfigs.SASL_SERVER_MAX_RECEIVE_SIZE_DOC; - public static final int SASL_SERVER_MAX_RECEIVE_SIZE_DEFAULT = BrokerSecurityConfigs.DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE; - - public final static String SECURITY_PROVIDER_CLASS_CONFIG = SecurityConfig.SECURITY_PROVIDERS_CONFIG; - public final static String SECURITY_PROVIDERS_DOC = SecurityConfig.SECURITY_PROVIDERS_DOC; -} diff --git a/server/src/main/java/org/apache/kafka/server/config/ServerConfigs.java b/server/src/main/java/org/apache/kafka/server/config/ServerConfigs.java new file mode 100644 index 0000000000000..a0b8cea05c3b8 --- /dev/null +++ b/server/src/main/java/org/apache/kafka/server/config/ServerConfigs.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.config; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.server.authorizer.Authorizer; +public class ServerConfigs { + /** ********* General Configuration ***********/ + public static final String RESERVED_BROKER_MAX_ID_CONFIG = "reserved.broker.max.id"; + public static final int RESERVED_BROKER_MAX_ID_DEFAULT = 1000; + public static final String RESERVED_BROKER_MAX_ID_DOC = "Max number that can be used for a broker.id"; + + public static final String BROKER_ID_GENERATION_ENABLE_CONFIG = "broker.id.generation.enable"; + public static final boolean BROKER_ID_GENERATION_ENABLE_DEFAULT = true; + public static final String BROKER_ID_GENERATION_ENABLE_DOC = "Enable automatic broker id generation on the server. When enabled the value configured for " + RESERVED_BROKER_MAX_ID_CONFIG + " should be reviewed."; + + + public static final String BROKER_ID_CONFIG = "broker.id"; + public static final int BROKER_ID_DEFAULT = -1; + public static final String BROKER_ID_DOC = "The broker id for this server. If unset, a unique broker id will be generated." + + "To avoid conflicts between ZooKeeper generated broker id's and user configured broker id's, generated broker ids " + + "start from " + RESERVED_BROKER_MAX_ID_CONFIG + " + 1."; + + public static final String MESSAGE_MAX_BYTES_CONFIG = "message.max.bytes"; + public static final String MESSAGE_MAX_BYTES_DOC = TopicConfig.MAX_MESSAGE_BYTES_DOC + + "This can be set per topic with the topic level " + TopicConfig.MAX_MESSAGE_BYTES_CONFIG + " config."; + + public static final String NUM_NETWORK_THREADS_CONFIG = "num.network.threads"; + public static final int NUM_NETWORK_THREADS_DEFAULT = 3; + public static final String NUM_NETWORK_THREADS_DOC = "The number of threads that the server uses for receiving requests from the network and sending responses to the network. Noted: each listener (except for controller listener) creates its own thread pool."; + + public static final String NUM_IO_THREADS_CONFIG = "num.io.threads"; + public static final int NUM_IO_THREADS_DEFAULT = 8; + public static final String NUM_IO_THREADS_DOC = "The number of threads that the server uses for processing requests, which may include disk I/O"; + + public static final String BACKGROUND_THREADS_CONFIG = "background.threads"; + public static final int BACKGROUND_THREADS_DEFAULT = 10; + public static final String BACKGROUND_THREADS_DOC = "The number of threads to use for various background processing tasks"; + + public static final String NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG = "num.replica.alter.log.dirs.threads"; + public static final String NUM_REPLICA_ALTER_LOG_DIRS_THREADS_DOC = "The number of threads that can move replicas between log directories, which may include disk I/O"; + + public static final String QUEUED_MAX_REQUESTS_CONFIG = "queued.max.requests"; + public static final int QUEUED_MAX_REQUESTS_DEFAULT = 500; + public static final String QUEUED_MAX_REQUESTS_DOC = "The number of queued requests allowed for data-plane, before blocking the network threads"; + + public static final String QUEUED_MAX_BYTES_CONFIG = "queued.max.request.bytes"; + public static final int QUEUED_MAX_REQUEST_BYTES_DEFAULT = -1; + public static final String QUEUED_MAX_REQUEST_BYTES_DOC = "The number of queued bytes allowed before no more requests are read"; + + public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG; + public static final int REQUEST_TIMEOUT_MS_DEFAULT = 30000; + public static final String REQUEST_TIMEOUT_MS_DOC = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC; + + public static final String SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG; + public static final long DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS; + public static final String SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC; + + public static final String SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG; + public static final long SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS; + public static final String SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC; + + public static final String DELETE_TOPIC_ENABLE_CONFIG = "delete.topic.enable"; + public static final boolean DELETE_TOPIC_ENABLE_DEFAULT = true; + public static final String DELETE_TOPIC_ENABLE_DOC = "Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off"; + + public static final String COMPRESSION_TYPE_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_TYPE_CONFIG); + public static final String COMPRESSION_TYPE_DOC = "Specify the final compression type for a given topic. This configuration accepts the standard compression codecs " + + "('gzip', 'snappy', 'lz4', 'zstd'). It additionally accepts 'uncompressed' which is equivalent to no compression; and " + + "'producer' which means retain the original compression codec set by the producer."; + + public static final String COMPRESSION_GZIP_LEVEL_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG); + public static final String COMPRESSION_GZIP_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to 'gzip'."; + public static final String COMPRESSION_LZ4_LEVEL_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG); + public static final String COMPRESSION_LZ4_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to 'lz4'."; + public static final String COMPRESSION_ZSTD_LEVEL_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG); + public static final String COMPRESSION_ZSTD_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to 'zstd'."; + + /***************** rack configuration *************/ + public static final String BROKER_RACK_CONFIG = "broker.rack"; + public static final String BROKER_RACK_DOC = "Rack of the broker. This will be used in rack aware replication assignment for fault tolerance. Examples: RACK1, us-east-1d"; + + /** ********* Controlled shutdown configuration ***********/ + public static final String CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG = "controlled.shutdown.max.retries"; + public static final int CONTROLLED_SHUTDOWN_MAX_RETRIES_DEFAULT = 3; + public static final String CONTROLLED_SHUTDOWN_MAX_RETRIES_DOC = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens"; + + public static final String CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG = "controlled.shutdown.retry.backoff.ms"; + public static final int CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_DEFAULT = 5000; + public static final String CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_DOC = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying."; + + public static final String CONTROLLED_SHUTDOWN_ENABLE_CONFIG = "controlled.shutdown.enable"; + public static final boolean CONTROLLED_SHUTDOWN_ENABLE_DEFAULT = true; + public static final String CONTROLLED_SHUTDOWN_ENABLE_DOC = "Enable controlled shutdown of the server."; + + /** ********* Fetch Configuration **************/ + public static final String MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_CONFIG = "max.incremental.fetch.session.cache.slots"; + public static final int MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_DEFAULT = 1000; + public static final String MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_DOC = "The maximum number of total incremental fetch sessions that we will maintain. FetchSessionCache is sharded into 8 shards and the limit is equally divided among all shards. Sessions are allocated to each shard in round-robin. Only entries within a shard are considered eligible for eviction."; + + public static final String FETCH_MAX_BYTES_CONFIG = "fetch.max.bytes"; + public static final int FETCH_MAX_BYTES_DEFAULT = 55 * 1024 * 1024; + public static final String FETCH_MAX_BYTES_DOC = "The maximum number of bytes we will return for a fetch request. Must be at least 1024."; + + /** ********* Request Limit Configuration **************/ + public static final String MAX_REQUEST_PARTITION_SIZE_LIMIT_CONFIG = "max.request.partition.size.limit"; + public static final int MAX_REQUEST_PARTITION_SIZE_LIMIT_DEFAULT = 2000; + public static final String MAX_REQUEST_PARTITION_SIZE_LIMIT_DOC = "The maximum number of partitions can be served in one request."; + + /** Internal Configurations **/ + public static final String UNSTABLE_API_VERSIONS_ENABLE_CONFIG = "unstable.api.versions.enable"; + public static final String UNSTABLE_FEATURE_VERSIONS_ENABLE_CONFIG = "unstable.feature.versions.enable"; + + /************* Authorizer Configuration ***********/ + public static final String AUTHORIZER_CLASS_NAME_CONFIG = "authorizer.class.name"; + public static final String AUTHORIZER_CLASS_NAME_DEFAULT = ""; + public static final String AUTHORIZER_CLASS_NAME_DOC = "The fully qualified name of a class that implements " + + Authorizer.class.getName() + " interface, which is used by the broker for authorization."; + public static final String EARLY_START_LISTENERS_CONFIG = "early.start.listeners"; + public static final String EARLY_START_LISTENERS_DOC = "A comma-separated list of listener names which may be started before the authorizer has finished " + + "initialization. This is useful when the authorizer is dependent on the cluster itself for bootstrapping, as is the case for " + + "the StandardAuthorizer (which stores ACLs in the metadata log.) By default, all listeners included in controller.listener.names " + + "will also be early start listeners. A listener should not appear in this list if it accepts external traffic."; +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 966d841fb44e6..d6cf615c781b3 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -100,6 +100,16 @@ public final class RemoteLogManagerConfig { "segments, fetch remote log indexes and clean up remote log segments."; public static final int DEFAULT_REMOTE_LOG_MANAGER_THREAD_POOL_SIZE = 10; + public static final String REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP = "remote.log.manager.copier.thread.pool.size"; + public static final String REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_DOC = "Size of the thread pool used in " + + "scheduling tasks to copy segments."; + public static final int DEFAULT_REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE = 10; + + public static final String REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP = "remote.log.manager.expiration.thread.pool.size"; + public static final String REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_DOC = "Size of the thread pool used in" + + " scheduling tasks to clean up remote log segments."; + public static final int DEFAULT_REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE = 10; + public static final String REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP = "remote.log.manager.task.interval.ms"; public static final String REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_DOC = "Interval at which remote log manager runs the scheduled tasks like copy " + "segments, and clean up remote log segments."; @@ -143,6 +153,38 @@ public final class RemoteLogManagerConfig { "less than or equal to `log.retention.bytes` value."; public static final Long DEFAULT_LOG_LOCAL_RETENTION_BYTES = -2L; + public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.copy.max.bytes.per.second"; + public static final String REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_DOC = "The maximum number of bytes that can be copied from local storage to remote storage per second. " + + "This is a global limit for all the partitions that are being copied from local storage to remote storage. " + + "The default value is Long.MAX_VALUE, which means there is no limit on the number of bytes that can be copied per second."; + public static final Long DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND = Long.MAX_VALUE; + + public static final String REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM_PROP = "remote.log.manager.copy.quota.window.num"; + public static final String REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM_DOC = "The number of samples to retain in memory for remote copy quota management. " + + "The default value is 11, which means there are 10 whole windows + 1 current window."; + public static final int DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM = 11; + + public static final String REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS_PROP = "remote.log.manager.copy.quota.window.size.seconds"; + public static final String REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS_DOC = "The time span of each sample for remote copy quota management. " + + "The default value is 1 second."; + public static final int DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS = 1; + + public static final String REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP = "remote.log.manager.fetch.max.bytes.per.second"; + public static final String REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_DOC = "The maximum number of bytes that can be fetched from remote storage to local storage per second. " + + "This is a global limit for all the partitions that are being fetched from remote storage to local storage. " + + "The default value is Long.MAX_VALUE, which means there is no limit on the number of bytes that can be fetched per second."; + public static final Long DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND = Long.MAX_VALUE; + + public static final String REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM_PROP = "remote.log.manager.fetch.quota.window.num"; + public static final String REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM_DOC = "The number of samples to retain in memory for remote fetch quota management. " + + "The default value is 11, which means there are 10 whole windows + 1 current window."; + public static final int DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM = 11; + + public static final String REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS_PROP = "remote.log.manager.fetch.quota.window.size.seconds"; + public static final String REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS_DOC = "The time span of each sample for remote fetch quota management. " + + "The default value is 1 second."; + public static final int DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS = 1; + public static final ConfigDef CONFIG_DEF = new ConfigDef(); static { @@ -209,6 +251,18 @@ public final class RemoteLogManagerConfig { atLeast(1), MEDIUM, REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_DOC) + .defineInternal(REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP, + INT, + DEFAULT_REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE, + atLeast(1), + MEDIUM, + REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_DOC) + .defineInternal(REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP, + INT, + DEFAULT_REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE, + atLeast(1), + MEDIUM, + REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_DOC) .define(REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP, LONG, DEFAULT_REMOTE_LOG_MANAGER_TASK_INTERVAL_MS, @@ -255,7 +309,43 @@ public final class RemoteLogManagerConfig { DEFAULT_LOG_LOCAL_RETENTION_BYTES, atLeast(DEFAULT_LOG_LOCAL_RETENTION_BYTES), MEDIUM, - LOG_LOCAL_RETENTION_BYTES_DOC); + LOG_LOCAL_RETENTION_BYTES_DOC) + .define(REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, + LONG, + DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, + atLeast(1), + MEDIUM, + REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_DOC) + .define(REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM_PROP, + INT, + DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM, + atLeast(1), + MEDIUM, + REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM_DOC) + .define(REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS_PROP, + INT, + DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS, + atLeast(1), + MEDIUM, + REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS_DOC) + .define(REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP, + LONG, + DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND, + atLeast(1), + MEDIUM, + REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_DOC) + .define(REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM_PROP, + INT, + DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM, + atLeast(1), + MEDIUM, + REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM_DOC) + .define(REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS_PROP, + INT, + DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS, + atLeast(1), + MEDIUM, + REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS_DOC); } private final boolean enableRemoteStorageSystem; @@ -265,6 +355,8 @@ public final class RemoteLogManagerConfig { private final String remoteLogMetadataManagerClassPath; private final long remoteLogIndexFileCacheTotalSizeBytes; private final int remoteLogManagerThreadPoolSize; + private final int remoteLogManagerCopierThreadPoolSize; + private final int remoteLogManagerExpirationThreadPoolSize; private final long remoteLogManagerTaskIntervalMs; private final long remoteLogManagerTaskRetryBackoffMs; private final long remoteLogManagerTaskRetryBackoffMaxMs; @@ -277,6 +369,12 @@ public final class RemoteLogManagerConfig { private final HashMap remoteLogMetadataManagerProps; private final String remoteLogMetadataManagerListenerName; private final int remoteLogMetadataCustomMetadataMaxBytes; + private final long remoteLogManagerCopyMaxBytesPerSecond; + private final int remoteLogManagerCopyNumQuotaSamples; + private final int remoteLogManagerCopyQuotaWindowSizeSeconds; + private final long remoteLogManagerFetchMaxBytesPerSecond; + private final int remoteLogManagerFetchNumQuotaSamples; + private final int remoteLogManagerFetchQuotaWindowSizeSeconds; public RemoteLogManagerConfig(AbstractConfig config) { this(config.getBoolean(REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP), @@ -287,6 +385,8 @@ public RemoteLogManagerConfig(AbstractConfig config) { config.getString(REMOTE_LOG_METADATA_MANAGER_LISTENER_NAME_PROP), config.getLong(REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP), config.getInt(REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_PROP), + config.getInt(REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP), + config.getInt(REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP), config.getLong(REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP), config.getLong(REMOTE_LOG_MANAGER_TASK_RETRY_BACK_OFF_MS_PROP), config.getLong(REMOTE_LOG_MANAGER_TASK_RETRY_BACK_OFF_MAX_MS_PROP), @@ -301,7 +401,13 @@ public RemoteLogManagerConfig(AbstractConfig config) { config.getString(REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX_PROP), config.getString(REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX_PROP) != null ? config.originalsWithPrefix(config.getString(REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX_PROP)) - : Collections.emptyMap()); + : Collections.emptyMap(), + config.getLong(REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP), + config.getInt(REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM_PROP), + config.getInt(REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS_PROP), + config.getLong(REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP), + config.getInt(REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM_PROP), + config.getInt(REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS_PROP)); } // Visible for testing @@ -313,6 +419,8 @@ public RemoteLogManagerConfig(boolean enableRemoteStorageSystem, String remoteLogMetadataManagerListenerName, long remoteLogIndexFileCacheTotalSizeBytes, int remoteLogManagerThreadPoolSize, + int remoteLogManagerCopierThreadPoolSize, + int remoteLogManagerExpirationThreadPoolSize, long remoteLogManagerTaskIntervalMs, long remoteLogManagerTaskRetryBackoffMs, long remoteLogManagerTaskRetryBackoffMaxMs, @@ -323,7 +431,13 @@ public RemoteLogManagerConfig(boolean enableRemoteStorageSystem, String remoteStorageManagerPrefix, Map remoteStorageManagerProps, /* properties having keys stripped out with remoteStorageManagerPrefix */ String remoteLogMetadataManagerPrefix, - Map remoteLogMetadataManagerProps /* properties having keys stripped out with remoteLogMetadataManagerPrefix */ + Map remoteLogMetadataManagerProps, /* properties having keys stripped out with remoteLogMetadataManagerPrefix */ + long remoteLogManagerCopyMaxBytesPerSecond, + int remoteLogManagerCopyNumQuotaSamples, + int remoteLogManagerCopyQuotaWindowSizeSeconds, + long remoteLogManagerFetchMaxBytesPerSecond, + int remoteLogManagerFetchNumQuotaSamples, + int remoteLogManagerFetchQuotaWindowSizeSeconds ) { this.enableRemoteStorageSystem = enableRemoteStorageSystem; this.remoteStorageManagerClassName = remoteStorageManagerClassName; @@ -332,6 +446,8 @@ public RemoteLogManagerConfig(boolean enableRemoteStorageSystem, this.remoteLogMetadataManagerClassPath = remoteLogMetadataManagerClassPath; this.remoteLogIndexFileCacheTotalSizeBytes = remoteLogIndexFileCacheTotalSizeBytes; this.remoteLogManagerThreadPoolSize = remoteLogManagerThreadPoolSize; + this.remoteLogManagerCopierThreadPoolSize = remoteLogManagerCopierThreadPoolSize; + this.remoteLogManagerExpirationThreadPoolSize = remoteLogManagerExpirationThreadPoolSize; this.remoteLogManagerTaskIntervalMs = remoteLogManagerTaskIntervalMs; this.remoteLogManagerTaskRetryBackoffMs = remoteLogManagerTaskRetryBackoffMs; this.remoteLogManagerTaskRetryBackoffMaxMs = remoteLogManagerTaskRetryBackoffMaxMs; @@ -344,6 +460,12 @@ public RemoteLogManagerConfig(boolean enableRemoteStorageSystem, this.remoteLogMetadataManagerProps = new HashMap<>(remoteLogMetadataManagerProps); this.remoteLogMetadataManagerListenerName = remoteLogMetadataManagerListenerName; this.remoteLogMetadataCustomMetadataMaxBytes = remoteLogMetadataCustomMetadataMaxBytes; + this.remoteLogManagerCopyMaxBytesPerSecond = remoteLogManagerCopyMaxBytesPerSecond; + this.remoteLogManagerCopyNumQuotaSamples = remoteLogManagerCopyNumQuotaSamples; + this.remoteLogManagerCopyQuotaWindowSizeSeconds = remoteLogManagerCopyQuotaWindowSizeSeconds; + this.remoteLogManagerFetchMaxBytesPerSecond = remoteLogManagerFetchMaxBytesPerSecond; + this.remoteLogManagerFetchNumQuotaSamples = remoteLogManagerFetchNumQuotaSamples; + this.remoteLogManagerFetchQuotaWindowSizeSeconds = remoteLogManagerFetchQuotaWindowSizeSeconds; } public boolean enableRemoteStorageSystem() { @@ -374,6 +496,14 @@ public int remoteLogManagerThreadPoolSize() { return remoteLogManagerThreadPoolSize; } + public int remoteLogManagerCopierThreadPoolSize() { + return remoteLogManagerCopierThreadPoolSize; + } + + public int remoteLogManagerExpirationThreadPoolSize() { + return remoteLogManagerExpirationThreadPoolSize; + } + public long remoteLogManagerTaskIntervalMs() { return remoteLogManagerTaskIntervalMs; } @@ -422,6 +552,31 @@ public Map remoteLogMetadataManagerProps() { return Collections.unmodifiableMap(remoteLogMetadataManagerProps); } + public long remoteLogManagerCopyMaxBytesPerSecond() { + return remoteLogManagerCopyMaxBytesPerSecond; + } + + public int remoteLogManagerCopyNumQuotaSamples() { + return remoteLogManagerCopyNumQuotaSamples; + } + + public int remoteLogManagerCopyQuotaWindowSizeSeconds() { + return remoteLogManagerCopyQuotaWindowSizeSeconds; + } + + public long remoteLogManagerFetchMaxBytesPerSecond() { + return remoteLogManagerFetchMaxBytesPerSecond; + } + + public int remoteLogManagerFetchNumQuotaSamples() { + return remoteLogManagerFetchNumQuotaSamples; + } + + public int remoteLogManagerFetchQuotaWindowSizeSeconds() { + return remoteLogManagerFetchQuotaWindowSizeSeconds; + } + + @Override public boolean equals(Object o) { if (this == o) return true; @@ -445,7 +600,13 @@ public boolean equals(Object o) { && Objects.equals(remoteStorageManagerProps, that.remoteStorageManagerProps) && Objects.equals(remoteLogMetadataManagerProps, that.remoteLogMetadataManagerProps) && Objects.equals(remoteStorageManagerPrefix, that.remoteStorageManagerPrefix) - && Objects.equals(remoteLogMetadataManagerPrefix, that.remoteLogMetadataManagerPrefix); + && Objects.equals(remoteLogMetadataManagerPrefix, that.remoteLogMetadataManagerPrefix) + && remoteLogManagerCopyMaxBytesPerSecond == that.remoteLogManagerCopyMaxBytesPerSecond + && remoteLogManagerCopyNumQuotaSamples == that.remoteLogManagerCopyNumQuotaSamples + && remoteLogManagerCopyQuotaWindowSizeSeconds == that.remoteLogManagerCopyQuotaWindowSizeSeconds + && remoteLogManagerFetchMaxBytesPerSecond == that.remoteLogManagerFetchMaxBytesPerSecond + && remoteLogManagerFetchNumQuotaSamples == that.remoteLogManagerFetchNumQuotaSamples + && remoteLogManagerFetchQuotaWindowSizeSeconds == that.remoteLogManagerFetchQuotaWindowSizeSeconds; } @Override @@ -455,7 +616,9 @@ public int hashCode() { remoteLogMetadataCustomMetadataMaxBytes, remoteLogIndexFileCacheTotalSizeBytes, remoteLogManagerThreadPoolSize, remoteLogManagerTaskIntervalMs, remoteLogManagerTaskRetryBackoffMs, remoteLogManagerTaskRetryBackoffMaxMs, remoteLogManagerTaskRetryJitter, remoteLogReaderThreads, remoteLogReaderMaxPendingTasks, remoteStorageManagerProps, remoteLogMetadataManagerProps, - remoteStorageManagerPrefix, remoteLogMetadataManagerPrefix); + remoteStorageManagerPrefix, remoteLogMetadataManagerPrefix, remoteLogManagerCopyMaxBytesPerSecond, + remoteLogManagerCopyNumQuotaSamples, remoteLogManagerCopyQuotaWindowSizeSeconds, remoteLogManagerFetchMaxBytesPerSecond, + remoteLogManagerFetchNumQuotaSamples, remoteLogManagerFetchQuotaWindowSizeSeconds); } public static void main(String[] args) { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchParams.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchParams.java index 700a8e40d5fe0..4af8cd1b39828 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchParams.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchParams.java @@ -22,6 +22,8 @@ import java.util.Objects; import java.util.Optional; +import static org.apache.kafka.common.requests.FetchRequest.FUTURE_LOCAL_REPLICA_ID; + public class FetchParams { public final short requestVersion; public final int replicaId; @@ -56,6 +58,10 @@ public boolean isFromFollower() { return FetchRequest.isValidBrokerId(replicaId); } + public boolean isFromFuture() { + return replicaId == FUTURE_LOCAL_REPLICA_ID; + } + public boolean isFromConsumer() { return FetchRequest.isConsumer(replicaId); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogOffsetMetadata.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogOffsetMetadata.java index 97e96928b2c4d..71f5f67c97217 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogOffsetMetadata.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogOffsetMetadata.java @@ -28,11 +28,10 @@ public final class LogOffsetMetadata { //TODO KAFKA-14484 remove once UnifiedLog has been moved to the storage module private static final long UNIFIED_LOG_UNKNOWN_OFFSET = -1L; - - public static final LogOffsetMetadata UNKNOWN_OFFSET_METADATA = new LogOffsetMetadata(-1L, 0L, 0); - private static final int UNKNOWN_FILE_POSITION = -1; + public static final LogOffsetMetadata UNKNOWN_OFFSET_METADATA = new LogOffsetMetadata(-1L, UNIFIED_LOG_UNKNOWN_OFFSET, UNKNOWN_FILE_POSITION); + public final long messageOffset; public final long segmentBaseOffset; public final int relativePositionInSegment; @@ -51,21 +50,22 @@ public LogOffsetMetadata(long messageOffset, // check if this offset is already on an older segment compared with the given offset public boolean onOlderSegment(LogOffsetMetadata that) { - if (messageOffsetOnly()) - throw new KafkaException(this + " cannot compare its segment info with " + that + " since it only has message offset info"); - + if (messageOffsetOnly() || that.messageOffsetOnly()) + return false; return this.segmentBaseOffset < that.segmentBaseOffset; } // check if this offset is on the same segment with the given offset - private boolean onSameSegment(LogOffsetMetadata that) { + public boolean onSameSegment(LogOffsetMetadata that) { + if (messageOffsetOnly() || that.messageOffsetOnly()) + return false; return this.segmentBaseOffset == that.segmentBaseOffset; } // compute the number of bytes between this offset to the given offset // if they are on the same segment and this offset precedes the given offset public int positionDiff(LogOffsetMetadata that) { - if (messageOffsetOnly()) + if (messageOffsetOnly() || that.messageOffsetOnly()) throw new KafkaException(this + " cannot compare its segment position with " + that + " since it only has message offset info"); if (!onSameSegment(that)) throw new KafkaException(this + " cannot compare its segment position with " + that + " since they are not on the same segment"); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java index 104eb9a18b349..969fa8b1db455 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java @@ -398,7 +398,7 @@ LogOffsetPosition translateOffset(long offset, int startingFilePosition) throws /** * Equivalent to {@code read(startOffset, maxSize, size())}. * - * See {@link #read(long, int, long, boolean)} for details. + * See {@link #read(long, int, Optional, boolean)} for details. */ public FetchDataInfo read(long startOffset, int maxSize) throws IOException { return read(startOffset, maxSize, size()); @@ -407,10 +407,10 @@ public FetchDataInfo read(long startOffset, int maxSize) throws IOException { /** * Equivalent to {@code read(startOffset, maxSize, maxPosition, false)}. * - * See {@link #read(long, int, long, boolean)} for details. + * See {@link #read(long, int, Optional, boolean)} for details. */ public FetchDataInfo read(long startOffset, int maxSize, long maxPosition) throws IOException { - return read(startOffset, maxSize, maxPosition, false); + return read(startOffset, maxSize, Optional.of(maxPosition), false); } /** @@ -421,13 +421,13 @@ public FetchDataInfo read(long startOffset, int maxSize, long maxPosition) throw * * @param startOffset A lower bound on the first offset to include in the message set we read * @param maxSize The maximum number of bytes to include in the message set we read - * @param maxPosition The maximum position in the log segment that should be exposed for read + * @param maxPositionOpt The maximum position in the log segment that should be exposed for read * @param minOneMessage If this is true, the first message will be returned even if it exceeds `maxSize` (if one exists) * * @return The fetched data and the offset metadata of the first message whose offset is >= startOffset, * or null if the startOffset is larger than the largest offset in this log */ - public FetchDataInfo read(long startOffset, int maxSize, long maxPosition, boolean minOneMessage) throws IOException { + public FetchDataInfo read(long startOffset, int maxSize, Optional maxPositionOpt, boolean minOneMessage) throws IOException { if (maxSize < 0) throw new IllegalArgumentException("Invalid max size " + maxSize + " for log read from segment " + log); @@ -444,12 +444,14 @@ public FetchDataInfo read(long startOffset, int maxSize, long maxPosition, boole if (minOneMessage) adjustedMaxSize = Math.max(maxSize, startOffsetAndSize.size); - // return a log segment but with zero size in the case below - if (adjustedMaxSize == 0) + // return empty records in the fetch-data-info when: + // 1. adjustedMaxSize is 0 (or) + // 2. maxPosition to read is unavailable + if (adjustedMaxSize == 0 || !maxPositionOpt.isPresent()) return new FetchDataInfo(offsetMetadata, MemoryRecords.EMPTY); // calculate the length of the message set to read based on whether or not they gave us a maxOffset - int fetchSize = Math.min((int) (maxPosition - startPosition), adjustedMaxSize); + int fetchSize = Math.min((int) (maxPositionOpt.get() - startPosition), adjustedMaxSize); return new FetchDataInfo(offsetMetadata, log.slice(startPosition, fetchSize), adjustedMaxSize < startOffsetAndSize.size, Optional.empty()); @@ -717,9 +719,9 @@ public long timeWaitedForRoll(long now, long messageTimestamp) { } /** - * @return the first batch timestamp if the timestamp is available. Otherwise return Long.MaxValue + * @return the first batch timestamp if the timestamp is available. Otherwise, return Long.MaxValue */ - long getFirstBatchTimestamp() { + public long getFirstBatchTimestamp() { loadFirstBatchTimestamp(); OptionalLong timestamp = rollingBasedTimestamp; if (timestamp.isPresent() && timestamp.getAsLong() >= 0) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegments.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegments.java index 6e2af251c0f81..580199f98c631 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegments.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegments.java @@ -141,7 +141,7 @@ public int numberOfSegments() { * @return the base offsets of all segments */ public Collection baseOffsets() { - return values().stream().map(s -> s.baseOffset()).collect(Collectors.toList()); + return values().stream().map(LogSegment::baseOffset).collect(Collectors.toList()); } /** @@ -219,7 +219,7 @@ private Optional> floorEntry(long offset) { * This method is thread-safe. */ public Optional floorSegment(long offset) { - return floorEntry(offset).map(e -> e.getValue()); + return floorEntry(offset).map(Map.Entry::getValue); } /** @@ -239,7 +239,7 @@ private Optional> lowerEntry(long offset) { * This method is thread-safe. */ public Optional lowerSegment(long offset) { - return lowerEntry(offset).map(e -> e.getValue()); + return lowerEntry(offset).map(Map.Entry::getValue); } /** @@ -259,7 +259,7 @@ public Optional> higherEntry(long offset) { * This method is thread-safe. */ public Optional higherSegment(long offset) { - return higherEntry(offset).map(e -> e.getValue()); + return higherEntry(offset).map(Map.Entry::getValue); } /** @@ -277,17 +277,15 @@ public Optional> firstEntry() { * This method is thread-safe. */ public Optional firstSegment() { - return firstEntry().map(s -> s.getValue()); + return firstEntry().map(Map.Entry::getValue); } /** * @return the base offset of the log segment associated with the smallest offset, if it exists */ public OptionalLong firstSegmentBaseOffset() { - Optional first = firstSegment(); - if (first.isPresent()) - return OptionalLong.of(first.get().baseOffset()); - return OptionalLong.empty(); + return firstSegment().map(logSegment -> OptionalLong.of(logSegment.baseOffset())) + .orElseGet(OptionalLong::empty); } /** @@ -305,7 +303,7 @@ public Optional> lastEntry() { * This method is thread-safe. */ public Optional lastSegment() { - return lastEntry().map(e -> e.getValue()); + return lastEntry().map(Map.Entry::getValue); } /** @@ -346,10 +344,6 @@ public Collection filter(Predicate predicate) { * @return Sum of the log segments' sizes (in bytes) */ public static long sizeInBytes(Collection segments) { - return segments.stream().mapToLong(s -> s.size()).sum(); - } - - public static Collection getFirstBatchTimestampForSegments(Collection segments) { - return segments.stream().map(s -> s.getFirstBatchTimestamp()).collect(Collectors.toList()); + return segments.stream().mapToLong(LogSegment::size).sum(); } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java index da56ddaccc864..125730e410c0c 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java @@ -176,11 +176,6 @@ private void addProducerId(long producerId, ProducerStateEntry entry) { producerIdCount = producers.size(); } - private void removeProducerIds(List keys) { - keys.forEach(producers::remove); - producerIdCount = producers.size(); - } - private void clearProducerIds() { producers.clear(); producerIdCount = 0; @@ -369,17 +364,12 @@ private boolean isProducerExpired(long currentTimeMs, ProducerStateEntry produce * Also expire any verification state entries that are lingering as unverified. */ public void removeExpiredProducers(long currentTimeMs) { - List keys = producers.entrySet().stream() - .filter(entry -> isProducerExpired(currentTimeMs, entry.getValue())) - .map(Map.Entry::getKey) - .collect(Collectors.toList()); - removeProducerIds(keys); - - List verificationKeys = verificationStates.entrySet().stream() - .filter(entry -> currentTimeMs - entry.getValue().timestamp() >= producerStateManagerConfig.producerIdExpirationMs()) - .map(Map.Entry::getKey) - .collect(Collectors.toList()); - verificationKeys.forEach(verificationStates::remove); + producers.entrySet().removeIf(entry -> isProducerExpired(currentTimeMs, entry.getValue())); + producerIdCount = producers.size(); + + verificationStates.entrySet().removeIf(entry -> + (currentTimeMs - entry.getValue().timestamp()) >= producerStateManagerConfig.producerIdExpirationMs() + ); } /** diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerHarness.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerHarness.java index a063fa8820a82..7af78e750a84f 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerHarness.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerHarness.java @@ -53,12 +53,6 @@ public void initialize(Set topicIdPartitions, initializeRemoteLogMetadataManager(topicIdPartitions, startConsumerThread, RemoteLogMetadataTopicPartitioner::new, remotePartitionMetadataStoreSupplier); } - public void initializeRemoteLogMetadataManager(Set topicIdPartitions, - boolean startConsumerThread, - Function remoteLogMetadataTopicPartitioner) { - initializeRemoteLogMetadataManager(topicIdPartitions, startConsumerThread, remoteLogMetadataTopicPartitioner, RemotePartitionMetadataStore::new); - } - public void initializeRemoteLogMetadataManager(Set topicIdPartitions, boolean startConsumerThread, Function remoteLogMetadataTopicPartitioner, @@ -70,6 +64,7 @@ public void initializeRemoteLogMetadataManager(Set topicIdPart .startConsumerThread(startConsumerThread) .remoteLogMetadataTopicPartitioner(remoteLogMetadataTopicPartitioner) .remotePartitionMetadataStore(remotePartitionMetadataStoreSupplier) + .overrideRemoteLogMetadataManagerProps(overrideRemoteLogMetadataManagerProps()) .build(); } diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java index c599259ed9416..84b98dcb5be1d 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java @@ -16,6 +16,11 @@ */ package org.apache.kafka.server.log.remote.metadata.storage; +import kafka.test.ClusterInstance; +import kafka.test.annotation.ClusterTest; +import kafka.test.junit.ClusterTestExtensions; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; @@ -24,139 +29,99 @@ import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import scala.collection.JavaConverters; -import scala.collection.Seq; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.extension.ExtendWith; -import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.LOG_DIR; -@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters +@ExtendWith(value = ClusterTestExtensions.class) +@Tag("integration") public class TopicBasedRemoteLogMetadataManagerRestartTest { private static final int SEG_SIZE = 1024 * 1024; private final Time time = new MockTime(1); private final String logDir = TestUtils.tempDirectory("_rlmm_segs_").getAbsolutePath(); + private final ClusterInstance clusterInstance; - private TopicBasedRemoteLogMetadataManagerHarness remoteLogMetadataManagerHarness; - - @BeforeEach - public void setup() { - // Start the cluster and initialize TopicBasedRemoteLogMetadataManager. - remoteLogMetadataManagerHarness = new TopicBasedRemoteLogMetadataManagerHarness() { - protected Map overrideRemoteLogMetadataManagerProps() { - Map props = new HashMap<>(); - props.put(LOG_DIR, logDir); - return props; - } - }; - remoteLogMetadataManagerHarness.initialize(Collections.emptySet(), true); + TopicBasedRemoteLogMetadataManagerRestartTest(ClusterInstance clusterInstance) { // Constructor injections + this.clusterInstance = clusterInstance; } - private void startTopicBasedRemoteLogMetadataManagerHarness(boolean startConsumerThread) { - remoteLogMetadataManagerHarness.initializeRemoteLogMetadataManager(Collections.emptySet(), startConsumerThread, RemoteLogMetadataTopicPartitioner::new); + private TopicBasedRemoteLogMetadataManager createTopicBasedRemoteLogMetadataManager() { + return RemoteLogMetadataManagerTestUtils.builder() + .topicIdPartitions(Collections.emptySet()) + .bootstrapServers(clusterInstance.bootstrapServers()) + .startConsumerThread(true) + .remoteLogMetadataTopicPartitioner(RemoteLogMetadataTopicPartitioner::new) + .overrideRemoteLogMetadataManagerProps(Collections.singletonMap(LOG_DIR, logDir)) + .build(); } - @AfterEach - public void teardown() throws IOException { - if (remoteLogMetadataManagerHarness != null) { - remoteLogMetadataManagerHarness.close(); - } - } - - private void stopTopicBasedRemoteLogMetadataManagerHarness() { - remoteLogMetadataManagerHarness.closeRemoteLogMetadataManager(); - } - - private TopicBasedRemoteLogMetadataManager topicBasedRlmm() { - return remoteLogMetadataManagerHarness.remoteLogMetadataManager(); - } - - @Test + @ClusterTest(brokers = 3) public void testRLMMAPIsAfterRestart() throws Exception { // Create topics. String leaderTopic = "new-leader"; - HashMap> assignedLeaderTopicReplicas = new HashMap<>(); - List leaderTopicReplicas = new ArrayList<>(); - // Set broker id 0 as the first entry which is taken as the leader. - leaderTopicReplicas.add(0); - leaderTopicReplicas.add(1); - leaderTopicReplicas.add(2); - assignedLeaderTopicReplicas.put(0, JavaConverters.asScalaBuffer(leaderTopicReplicas)); - remoteLogMetadataManagerHarness.createTopicWithAssignment( - leaderTopic, JavaConverters.mapAsScalaMap(assignedLeaderTopicReplicas), - remoteLogMetadataManagerHarness.listenerName()); - String followerTopic = "new-follower"; - HashMap> assignedFollowerTopicReplicas = new HashMap<>(); - List followerTopicReplicas = new ArrayList<>(); - // Set broker id 1 as the first entry which is taken as the leader. - followerTopicReplicas.add(1); - followerTopicReplicas.add(2); - followerTopicReplicas.add(0); - assignedFollowerTopicReplicas.put(0, JavaConverters.asScalaBuffer(followerTopicReplicas)); - remoteLogMetadataManagerHarness.createTopicWithAssignment(followerTopic, - JavaConverters.mapAsScalaMap(assignedFollowerTopicReplicas), - remoteLogMetadataManagerHarness.listenerName()); + try (Admin admin = clusterInstance.createAdminClient()) { + // Set broker id 0 as the first entry which is taken as the leader. + NewTopic newLeaderTopic = new NewTopic(leaderTopic, Collections.singletonMap(0, Arrays.asList(0, 1, 2))); + // Set broker id 1 as the first entry which is taken as the leader. + NewTopic newFollowerTopic = new NewTopic(followerTopic, Collections.singletonMap(0, Arrays.asList(1, 2, 0))); + admin.createTopics(Arrays.asList(newLeaderTopic, newFollowerTopic)).all().get(); + } + clusterInstance.waitForTopic(leaderTopic, 1); + clusterInstance.waitForTopic(followerTopic, 1); final TopicIdPartition leaderTopicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition(leaderTopic, 0)); final TopicIdPartition followerTopicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition(followerTopic, 0)); - - // Register these partitions to RLMM. - topicBasedRlmm().onPartitionLeadershipChanges(Collections.singleton(leaderTopicIdPartition), Collections.singleton(followerTopicIdPartition)); - - // Add segments for these partitions, but they are not available as they have not yet been subscribed. RemoteLogSegmentMetadata leaderSegmentMetadata = new RemoteLogSegmentMetadata( new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), 0, 100, -1L, 0, time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L)); - topicBasedRlmm().addRemoteLogSegmentMetadata(leaderSegmentMetadata).get(); - RemoteLogSegmentMetadata followerSegmentMetadata = new RemoteLogSegmentMetadata( new RemoteLogSegmentId(followerTopicIdPartition, Uuid.randomUuid()), 0, 100, -1L, 0, time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L)); - topicBasedRlmm().addRemoteLogSegmentMetadata(followerSegmentMetadata).get(); - - // Stop TopicBasedRemoteLogMetadataManager only. - stopTopicBasedRemoteLogMetadataManagerHarness(); - // Start TopicBasedRemoteLogMetadataManager - startTopicBasedRemoteLogMetadataManagerHarness(true); + try (TopicBasedRemoteLogMetadataManager topicBasedRemoteLogMetadataManager = createTopicBasedRemoteLogMetadataManager()) { + // Register these partitions to RemoteLogMetadataManager. + topicBasedRemoteLogMetadataManager.onPartitionLeadershipChanges( + Collections.singleton(leaderTopicIdPartition), Collections.singleton(followerTopicIdPartition)); - // Register these partitions to RLMM, which loads the respective metadata snapshots. - topicBasedRlmm().onPartitionLeadershipChanges( - Collections.singleton(leaderTopicIdPartition), Collections.singleton(followerTopicIdPartition)); - - // Check for the stored entries from the earlier run. - TestUtils.waitForCondition(() -> - TestUtils.sameElementsWithoutOrder(Collections.singleton(leaderSegmentMetadata).iterator(), - topicBasedRlmm().listRemoteLogSegments(leaderTopicIdPartition)), - "Remote log segment metadata not available"); - TestUtils.waitForCondition(() -> - TestUtils.sameElementsWithoutOrder(Collections.singleton(followerSegmentMetadata).iterator(), - topicBasedRlmm().listRemoteLogSegments(followerTopicIdPartition)), - "Remote log segment metadata not available"); - // Add one more segment - RemoteLogSegmentMetadata leaderSegmentMetadata2 = new RemoteLogSegmentMetadata( - new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), - 101, 200, -1L, 0, - time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 101L)); - topicBasedRlmm().addRemoteLogSegmentMetadata(leaderSegmentMetadata2).get(); + // Add segments for these partitions, but they are not available as they have not yet been subscribed. + topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(leaderSegmentMetadata).get(); + topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(followerSegmentMetadata).get(); + } - // Check that both the stored segment and recently added segment are available. - Assertions.assertTrue(TestUtils.sameElementsWithoutOrder(Arrays.asList(leaderSegmentMetadata, leaderSegmentMetadata2).iterator(), - topicBasedRlmm().listRemoteLogSegments(leaderTopicIdPartition))); + try (TopicBasedRemoteLogMetadataManager topicBasedRemoteLogMetadataManager = createTopicBasedRemoteLogMetadataManager()) { + // Register these partitions to RemoteLogMetadataManager, which loads the respective metadata snapshots. + topicBasedRemoteLogMetadataManager.onPartitionLeadershipChanges( + Collections.singleton(leaderTopicIdPartition), Collections.singleton(followerTopicIdPartition)); + + // Check for the stored entries from the earlier run. + TestUtils.waitForCondition(() -> + TestUtils.sameElementsWithoutOrder(Collections.singleton(leaderSegmentMetadata).iterator(), + topicBasedRemoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition)), + "Remote log segment metadata not available"); + TestUtils.waitForCondition(() -> + TestUtils.sameElementsWithoutOrder(Collections.singleton(followerSegmentMetadata).iterator(), + topicBasedRemoteLogMetadataManager.listRemoteLogSegments(followerTopicIdPartition)), + "Remote log segment metadata not available"); + // Add one more segment + RemoteLogSegmentMetadata leaderSegmentMetadata2 = new RemoteLogSegmentMetadata( + new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), + 101, 200, -1L, 0, + time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 101L)); + topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(leaderSegmentMetadata2).get(); + + // Check that both the stored segment and recently added segment are available. + Assertions.assertTrue(TestUtils.sameElementsWithoutOrder(Arrays.asList(leaderSegmentMetadata, leaderSegmentMetadata2).iterator(), + topicBasedRemoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition))); + } } -} \ No newline at end of file +} diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfigTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfigTest.java index 8bef2e3de79e0..4e3c2fc26cb66 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfigTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfigTest.java @@ -47,8 +47,9 @@ public void testValidConfigs(boolean useDefaultRemoteLogMetadataManagerClass) { RemoteLogManagerConfig expectedRemoteLogManagerConfig = new RemoteLogManagerConfig(true, "dummy.remote.storage.class", "dummy.remote.storage.class.path", remoteLogMetadataManagerClass, "dummy.remote.log.metadata.class.path", - "listener.name", 1024 * 1024L, 1, 60000L, 100L, 60000L, 0.3, 10, 100, 100, - rsmPrefix, rsmProps, rlmmPrefix, rlmmProps); + "listener.name", 1024 * 1024L, 1, 1, 1, 60000L, 100L, 60000L, 0.3, 10, 100, 100, + rsmPrefix, rsmProps, rlmmPrefix, rlmmProps, Long.MAX_VALUE, 11, 1, + Long.MAX_VALUE, 11, 1); Map props = extractProps(expectedRemoteLogManagerConfig); rsmProps.forEach((k, v) -> props.put(rsmPrefix + k, v)); @@ -80,6 +81,10 @@ private Map extractProps(RemoteLogManagerConfig remoteLogManager remoteLogManagerConfig.remoteLogIndexFileCacheTotalSizeBytes()); props.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_PROP, remoteLogManagerConfig.remoteLogManagerThreadPoolSize()); + props.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP, + remoteLogManagerConfig.remoteLogManagerCopierThreadPoolSize()); + props.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP, + remoteLogManagerConfig.remoteLogManagerExpirationThreadPoolSize()); props.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP, remoteLogManagerConfig.remoteLogManagerTaskIntervalMs()); props.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_TASK_RETRY_BACK_OFF_MS_PROP, diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogOffsetMetadataTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogOffsetMetadataTest.java new file mode 100644 index 0000000000000..0f1da1e415300 --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogOffsetMetadataTest.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.storage.internals.log; + +import org.apache.kafka.common.KafkaException; +import org.junit.jupiter.api.Test; + +import static org.apache.kafka.storage.internals.log.LogOffsetMetadata.UNKNOWN_OFFSET_METADATA; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class LogOffsetMetadataTest { + + @Test + void testOnOlderSegment() { + LogOffsetMetadata metadata1 = new LogOffsetMetadata(1L, 0L, 1); + LogOffsetMetadata metadata2 = new LogOffsetMetadata(5L, 4L, 2); + LogOffsetMetadata messageOnlyMetadata = new LogOffsetMetadata(1L); + assertFalse(UNKNOWN_OFFSET_METADATA.onOlderSegment(UNKNOWN_OFFSET_METADATA)); + assertFalse(metadata1.onOlderSegment(messageOnlyMetadata)); + assertFalse(messageOnlyMetadata.onOlderSegment(metadata1)); + assertFalse(metadata1.onOlderSegment(metadata1)); + assertFalse(metadata2.onOlderSegment(metadata1)); + assertTrue(metadata1.onOlderSegment(metadata2)); + } + + @Test + void testPositionDiff() { + LogOffsetMetadata metadata1 = new LogOffsetMetadata(1L); + LogOffsetMetadata metadata2 = new LogOffsetMetadata(5L, 0L, 5); + KafkaException exception = assertThrows(KafkaException.class, () -> metadata1.positionDiff(metadata2)); + assertTrue(exception.getMessage().endsWith("since it only has message offset info")); + + exception = assertThrows(KafkaException.class, () -> metadata2.positionDiff(metadata1)); + assertTrue(exception.getMessage().endsWith("since it only has message offset info")); + + LogOffsetMetadata metadata3 = new LogOffsetMetadata(15L, 10L, 5); + exception = assertThrows(KafkaException.class, () -> metadata3.positionDiff(metadata2)); + assertTrue(exception.getMessage().endsWith("since they are not on the same segment")); + + LogOffsetMetadata metadata4 = new LogOffsetMetadata(40L, 10L, 100); + assertEquals(95, metadata4.positionDiff(metadata3)); + } + + @Test + void testMessageOffsetOnly() { + LogOffsetMetadata metadata1 = new LogOffsetMetadata(1L); + LogOffsetMetadata metadata2 = new LogOffsetMetadata(1L, 0L, 1); + assertTrue(UNKNOWN_OFFSET_METADATA.messageOffsetOnly()); + assertFalse(metadata2.messageOffsetOnly()); + assertTrue(metadata1.messageOffsetOnly()); + } + + @Test + void testOnSameSegment() { + LogOffsetMetadata metadata1 = new LogOffsetMetadata(1L, 0L, 1); + LogOffsetMetadata metadata2 = new LogOffsetMetadata(5L, 4L, 2); + LogOffsetMetadata metadata3 = new LogOffsetMetadata(10L, 4L, 200); + assertFalse(metadata1.onSameSegment(metadata2)); + assertTrue(metadata2.onSameSegment(metadata3)); + + LogOffsetMetadata metadata4 = new LogOffsetMetadata(50); + LogOffsetMetadata metadata5 = new LogOffsetMetadata(100); + assertFalse(metadata4.onSameSegment(metadata5)); + } +} \ No newline at end of file diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestBuilder.java b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestBuilder.java index 6272d0e6863d8..f3f873b80d6f1 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestBuilder.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestBuilder.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.tiered.storage; +import org.apache.kafka.tiered.storage.actions.AlterLogDirAction; import org.apache.kafka.tiered.storage.actions.BounceBrokerAction; import org.apache.kafka.tiered.storage.actions.ConsumeAction; import org.apache.kafka.tiered.storage.actions.CreatePartitionsAction; @@ -55,6 +56,7 @@ import org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent; import org.apache.kafka.storage.internals.log.EpochEntry; +import java.io.FilenameFilter; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -290,6 +292,13 @@ public TieredStorageTestBuilder eraseBrokerStorage(Integer brokerId) { return this; } + public TieredStorageTestBuilder eraseBrokerStorage(Integer brokerId, + FilenameFilter filenameFilter, + boolean isStopped) { + actions.add(new EraseBrokerStorageAction(brokerId, filenameFilter, isStopped)); + return this; + } + public TieredStorageTestBuilder expectEmptyRemoteStorage(String topic, Integer partition) { TopicPartition topicPartition = new TopicPartition(topic, partition); @@ -313,6 +322,14 @@ public TieredStorageTestBuilder reassignReplica(String topic, return this; } + public TieredStorageTestBuilder alterLogDir(String topic, + Integer partition, + int replicaIds) { + TopicPartition topicPartition = new TopicPartition(topic, partition); + actions.add(new AlterLogDirAction(topicPartition, replicaIds)); + return this; + } + public TieredStorageTestBuilder expectUserTopicMappedToMetadataPartitions(String topic, List metadataPartitions) { actions.add(new ExpectUserTopicMappedToMetadataPartitionsAction(topic, metadataPartitions)); diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java index b3e2312145de5..059e33f3525b5 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java @@ -49,6 +49,7 @@ import scala.Function0; import scala.Function1; +import java.io.FilenameFilter; import java.io.IOException; import java.io.PrintStream; import java.util.ArrayList; @@ -259,8 +260,20 @@ public void start(int brokerId) { initContext(); } - public void eraseBrokerStorage(int brokerId) throws IOException { - localStorages.get(brokerId).eraseStorage(); + public void eraseBrokerStorage(int brokerId, + FilenameFilter filter, + boolean isStopped) throws IOException { + BrokerLocalStorage brokerLocalStorage; + if (isStopped) { + brokerLocalStorage = TieredStorageTestHarness.localStorages(harness.brokers()) + .stream() + .filter(bls -> bls.getBrokerId() == brokerId) + .findFirst() + .orElseThrow(() -> new IllegalArgumentException("No local storage found for broker " + brokerId)); + } else { + brokerLocalStorage = localStorages.get(brokerId); + } + brokerLocalStorage.eraseStorage(filter); } public TopicSpec topicSpec(String topicName) { diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java index 274bbf893b72a..1183383bea08a 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java @@ -154,7 +154,7 @@ public static List remoteStorageManagers(Seq br @SuppressWarnings("deprecation") public static List localStorages(Seq brokers) { return JavaConverters.seqAsJavaList(brokers).stream() - .map(b -> new BrokerLocalStorage(b.config().brokerId(), b.config().logDirs().head(), + .map(b -> new BrokerLocalStorage(b.config().brokerId(), JavaConverters.setAsJavaSet(b.config().logDirs().toSet()), STORAGE_WAIT_TIMEOUT_SEC)) .collect(Collectors.toList()); } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/AlterLogDirAction.java b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/AlterLogDirAction.java new file mode 100644 index 0000000000000..34f4c7d0892aa --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/AlterLogDirAction.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tiered.storage.actions; + +import org.apache.kafka.clients.admin.AlterReplicaLogDirsResult; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicPartitionReplica; +import org.apache.kafka.test.TestUtils; +import org.apache.kafka.tiered.storage.TieredStorageTestAction; +import org.apache.kafka.tiered.storage.TieredStorageTestContext; +import org.apache.kafka.tiered.storage.utils.BrokerLocalStorage; + +import java.io.File; +import java.io.PrintStream; +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public final class AlterLogDirAction implements TieredStorageTestAction { + + private final TopicPartition topicPartition; + private final int brokerId; + + public AlterLogDirAction(TopicPartition topicPartition, + int brokerId) { + this.topicPartition = topicPartition; + this.brokerId = brokerId; + } + + @Override + public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException, TimeoutException { + Optional localStorage = context.localStorages().stream().filter(storage -> storage.getBrokerId() == brokerId).findFirst(); + if (!localStorage.isPresent()) { + throw new IllegalArgumentException("cannot find local storage for this topic partition:" + topicPartition + " in this broker id:" + brokerId); + } + + Optional sourceDir = localStorage.get().getBrokerStorageDirectories().stream().filter(dir -> localStorage.get().dirContainsTopicPartition(topicPartition, dir)).findFirst(); + if (!sourceDir.isPresent()) { + throw new IllegalArgumentException("No log dir with topic partition:" + topicPartition + " in this broker id:" + brokerId); + } + Optional targetDir = localStorage.get().getBrokerStorageDirectories().stream().filter(dir -> !localStorage.get().dirContainsTopicPartition(topicPartition, dir)).findFirst(); + if (!targetDir.isPresent()) { + throw new IllegalArgumentException("No log dir without topic partition:" + topicPartition + " in this broker id:" + brokerId); + } + + // build alterReplicaLogDirs request content to move from sourceDir to targetDir + TopicPartitionReplica topicPartitionReplica = new TopicPartitionReplica(topicPartition.topic(), topicPartition.partition(), brokerId); + Map logDirs = Collections.singletonMap(topicPartitionReplica, targetDir.get().getAbsolutePath()); + + AlterReplicaLogDirsResult results = context.admin().alterReplicaLogDirs(logDirs); + results.values().get(topicPartitionReplica).get(30, TimeUnit.SECONDS); + + // wait until the topic partition folder disappears from source dir and appears in the target dir + TestUtils.waitForCondition(() -> localStorage.get().dirContainsTopicPartition(topicPartition, targetDir.get()) && + !localStorage.get().dirContainsTopicPartition(topicPartition, sourceDir.get()), + "Failed to alter dir:" + logDirs); + } + + @Override + public void describe(PrintStream output) { + output.print("alter dir for topic partition:" + topicPartition + " in this broker id:" + brokerId); + } +} diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/EraseBrokerStorageAction.java b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/EraseBrokerStorageAction.java index 3514630cb8273..1439a4b26704b 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/EraseBrokerStorageAction.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/EraseBrokerStorageAction.java @@ -19,24 +19,36 @@ import org.apache.kafka.tiered.storage.TieredStorageTestAction; import org.apache.kafka.tiered.storage.TieredStorageTestContext; +import java.io.FilenameFilter; import java.io.IOException; import java.io.PrintStream; public final class EraseBrokerStorageAction implements TieredStorageTestAction { private final int brokerId; + private final FilenameFilter filenameFilter; + private final boolean isStopped; public EraseBrokerStorageAction(int brokerId) { + this(brokerId, (dir, name) -> true, false); + } + + public EraseBrokerStorageAction(int brokerId, + FilenameFilter filenameFilter, + boolean isStopped) { this.brokerId = brokerId; + this.filenameFilter = filenameFilter; + this.isStopped = isStopped; } @Override public void doExecute(TieredStorageTestContext context) throws IOException { - context.eraseBrokerStorage(brokerId); + context.eraseBrokerStorage(brokerId, filenameFilter, isStopped); } @Override public void describe(PrintStream output) { - output.println("erase-broker-storage: " + brokerId); + output.println("erase-broker-storage: " + brokerId + ", isStopped: " + isStopped + + ", filenameFilter: " + filenameFilter); } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderAction.java b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderAction.java index f81cfcb5cf870..27a6818def90b 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderAction.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderAction.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.tiered.storage.actions; +import org.apache.kafka.clients.admin.AlterPartitionReassignmentsResult; import org.apache.kafka.tiered.storage.TieredStorageTestAction; import org.apache.kafka.tiered.storage.TieredStorageTestContext; import org.apache.kafka.clients.admin.NewPartitionReassignment; @@ -34,6 +35,8 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import static org.apache.kafka.tiered.storage.utils.TieredStorageTestUtils.describeTopic; @@ -53,7 +56,7 @@ public ExpectLeaderAction(TopicPartition topicPartition, } @Override - public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException { + public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException, TimeoutException { String topic = topicPartition.topic(); int partition = topicPartition.partition(); TestUtils.waitForCondition(() -> { @@ -88,7 +91,7 @@ public void describe(PrintStream output) { topicPartition, replicaId, electLeader); } - private void reassignPartition(TieredStorageTestContext context) throws ExecutionException, InterruptedException { + private void reassignPartition(TieredStorageTestContext context) throws ExecutionException, InterruptedException, TimeoutException { String topic = topicPartition.topic(); TopicPartitionInfo partitionInfo = describeTopic(context, topic) .partitions() @@ -104,6 +107,7 @@ private void reassignPartition(TieredStorageTestContext context) throws Executio Map> proposed = Collections.singletonMap(topicPartition, Optional.of(new NewPartitionReassignment(targetReplicas))); - context.admin().alterPartitionReassignments(proposed); + AlterPartitionReassignmentsResult result = context.admin().alterPartitionReassignments(proposed); + result.all().get(30, TimeUnit.MINUTES); } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/AlterLogDirTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/AlterLogDirTest.java new file mode 100644 index 0000000000000..d73ba53b677be --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/AlterLogDirTest.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tiered.storage.integration; + +import org.apache.kafka.tiered.storage.TieredStorageTestBuilder; +import org.apache.kafka.tiered.storage.TieredStorageTestHarness; +import org.apache.kafka.tiered.storage.specs.KeyValueSpec; + +import java.util.Arrays; +import java.util.Collections; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; + +public final class AlterLogDirTest extends TieredStorageTestHarness { + + @Override + public int brokerCount() { + return 2; + } + + @Override + protected void writeTestSpecifications(TieredStorageTestBuilder builder) { + final String topicB = "topicB"; + final int p0 = 0; + final int partitionCount = 1; + final int replicationFactor = 2; + final int maxBatchCountPerSegment = 1; + final boolean enableRemoteLogStorage = true; + final int broker0 = 0; + final int broker1 = 1; + + builder + // create topicB with 1 partition and 1 RF + .createTopic(topicB, partitionCount, replicationFactor, maxBatchCountPerSegment, + mkMap(mkEntry(p0, Arrays.asList(broker1, broker0))), enableRemoteLogStorage) + // send records to partition 0 + .expectSegmentToBeOffloaded(broker1, topicB, p0, 0, new KeyValueSpec("k0", "v0")) + .expectSegmentToBeOffloaded(broker1, topicB, p0, 1, new KeyValueSpec("k1", "v1")) + .expectEarliestLocalOffsetInLogDirectory(topicB, p0, 2L) + .produce(topicB, p0, new KeyValueSpec("k0", "v0"), new KeyValueSpec("k1", "v1"), + new KeyValueSpec("k2", "v2")) + // alter dir within the replica, we only expect one replicaId + .alterLogDir(topicB, p0, Collections.singletonList(broker0).get(0)) + // make sure the altered replica can still be elected as the leader + .expectLeader(topicB, p0, broker0, true) + // produce some more events and verify the earliest local offset + .expectEarliestLocalOffsetInLogDirectory(topicB, p0, 3L) + .produce(topicB, p0, new KeyValueSpec("k3", "v3")) + // consume from the beginning of the topic to read data from local and remote storage + .expectFetchFromTieredStorage(broker0, topicB, p0, 3) + .consume(topicB, p0, 0L, 4, 3); + } +} diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java new file mode 100644 index 0000000000000..556977e54978e --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tiered.storage.integration; + +import kafka.log.LogManager; +import kafka.server.ReplicaManager; +import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler; +import org.apache.kafka.tiered.storage.TieredStorageTestBuilder; +import org.apache.kafka.tiered.storage.TieredStorageTestHarness; +import org.apache.kafka.tiered.storage.specs.KeyValueSpec; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; + +public class FetchFromLeaderWithCorruptedCheckpointTest extends TieredStorageTestHarness { + + @Override + public int brokerCount() { + return 2; + } + + @Override + protected void writeTestSpecifications(TieredStorageTestBuilder builder) { + final Integer broker0 = 0; + final Integer broker1 = 1; + final String topicA = "topicA"; + final Integer p0 = 0; + final Integer partitionCount = 1; + final Integer replicationFactor = 2; + final Integer maxBatchCountPerSegment = 1; + final boolean enableRemoteLogStorage = true; + final Map> assignment = mkMap(mkEntry(p0, Arrays.asList(broker0, broker1))); + final List checkpointFiles = Arrays.asList( + ReplicaManager.HighWatermarkFilename(), + LogManager.RecoveryPointCheckpointFile(), + CleanShutdownFileHandler.CLEAN_SHUTDOWN_FILE_NAME); + + builder.createTopic(topicA, partitionCount, replicationFactor, maxBatchCountPerSegment, assignment, + enableRemoteLogStorage) + // send records to partition 0 + .expectSegmentToBeOffloaded(broker0, topicA, p0, 0, new KeyValueSpec("k0", "v0")) + .expectSegmentToBeOffloaded(broker0, topicA, p0, 1, new KeyValueSpec("k1", "v1")) + .expectEarliestLocalOffsetInLogDirectory(topicA, p0, 2L) + .produce(topicA, p0, new KeyValueSpec("k0", "v0"), new KeyValueSpec("k1", "v1"), + new KeyValueSpec("k2", "v2")) + .expectFetchFromTieredStorage(broker0, topicA, p0, 2) + .consume(topicA, p0, 0L, 3, 2) + // shutdown the brokers + .stop(broker1) + .stop(broker0) + // delete the checkpoint files + .eraseBrokerStorage(broker0, (dir, name) -> checkpointFiles.contains(name), true) + // start the broker first whose checkpoint files were deleted. + .start(broker0) + .start(broker1) + // send some records to partition 0 + // Note that the segment 2 gets offloaded for p0, but we cannot expect those events deterministically + // because the rlm-task-thread runs in background and this framework doesn't support it. + .expectSegmentToBeOffloaded(broker0, topicA, p0, 3, new KeyValueSpec("k3", "v3")) + .expectEarliestLocalOffsetInLogDirectory(topicA, p0, 4L) + .produce(topicA, p0, new KeyValueSpec("k3", "v3"), new KeyValueSpec("k4", "v4")) + .expectFetchFromTieredStorage(broker0, topicA, p0, 4) + .consume(topicA, p0, 0L, 5, 4); + + } +} diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/TransactionsWithTieredStoreTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/TransactionsWithTieredStoreTest.java index 1e9736084aaeb..207f9bd4e837f 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/TransactionsWithTieredStoreTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/TransactionsWithTieredStoreTest.java @@ -74,7 +74,7 @@ public Properties topicConfig() { public void maybeWaitForAtLeastOneSegmentUpload(scala.collection.Seq topicPartitions) { JavaConverters.seqAsJavaList(topicPartitions).forEach(topicPartition -> { List localStorages = JavaConverters.bufferAsJavaList(brokers()).stream() - .map(b -> new BrokerLocalStorage(b.config().brokerId(), b.config().logDirs().head(), STORAGE_WAIT_TIMEOUT_SEC)) + .map(b -> new BrokerLocalStorage(b.config().brokerId(), JavaConverters.setAsJavaSet(b.config().logDirs().toSet()), STORAGE_WAIT_TIMEOUT_SEC)) .collect(Collectors.toList()); localStorages .stream() diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/utils/BrokerLocalStorage.java b/storage/src/test/java/org/apache/kafka/tiered/storage/utils/BrokerLocalStorage.java index 2d26e43f8d7ee..639a464f3c910 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/utils/BrokerLocalStorage.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/utils/BrokerLocalStorage.java @@ -25,12 +25,14 @@ import org.apache.kafka.storage.internals.log.LogFileUtils; import java.io.File; +import java.io.FilenameFilter; import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; @@ -38,17 +40,17 @@ public final class BrokerLocalStorage { private final Integer brokerId; - private final File brokerStorageDirectory; + private final Set brokerStorageDirectories; private final Integer storageWaitTimeoutSec; private final int storagePollPeriodSec = 1; private final Time time = Time.SYSTEM; public BrokerLocalStorage(Integer brokerId, - String storageDirname, + Set storageDirnames, Integer storageWaitTimeoutSec) { this.brokerId = brokerId; - this.brokerStorageDirectory = new File(storageDirname); + this.brokerStorageDirectories = storageDirnames.stream().map(File::new).collect(Collectors.toSet()); this.storageWaitTimeoutSec = storageWaitTimeoutSec; } @@ -56,6 +58,10 @@ public Integer getBrokerId() { return brokerId; } + public Set getBrokerStorageDirectories() { + return brokerStorageDirectories; + } + /** * Wait until the first segment offset in Apache Kafka storage for the given topic-partition is * equal to the provided offset. @@ -141,7 +147,12 @@ private boolean isOffsetPresentInFirstLocalSegment(TopicPartition topicPartition if (offsetToSearch.equals(firstLogFileBaseOffset)) { return true; } - File partitionDir = new File(brokerStorageDirectory.getAbsolutePath(), topicPartition.toString()); + File logDir = brokerStorageDirectories.stream() + .filter(dir -> dirContainsTopicPartition(topicPartition, dir)) + .findFirst() + .orElseThrow(() -> new IllegalArgumentException(String.format("[BrokerId=%d] Directory for the topic-partition %s " + + "was not found", brokerId, topicPartition))); + File partitionDir = new File(logDir.getAbsolutePath(), topicPartition.toString()); File firstSegmentFile = new File(partitionDir.getAbsolutePath(), LogFileUtils.filenamePrefixFromOffset(firstLogFileBaseOffset) + LogFileUtils.LOG_FILE_SUFFIX); try (FileRecords fileRecords = FileRecords.open(firstSegmentFile, false)) { @@ -156,14 +167,16 @@ private boolean isOffsetPresentInFirstLocalSegment(TopicPartition topicPartition return false; } - public void eraseStorage() throws IOException { - for (File file : Objects.requireNonNull(brokerStorageDirectory.listFiles())) { - Utils.delete(file); + public void eraseStorage(FilenameFilter filter) throws IOException { + for (File brokerDir : brokerStorageDirectories) { + for (File file : Objects.requireNonNull(brokerDir.listFiles(filter))) { + Utils.delete(file); + } } } private OffsetHolder getEarliestLocalOffset(TopicPartition topicPartition) { - List partitionFiles = getTopicPartitionFiles(topicPartition); + List partitionFiles = getTopicPartitionFileNames(topicPartition); Optional firstLogFile = partitionFiles.stream() .filter(filename -> filename.endsWith(LogFileUtils.LOG_FILE_SUFFIX)) .sorted() @@ -175,8 +188,29 @@ private OffsetHolder getEarliestLocalOffset(TopicPartition topicPartition) { return new OffsetHolder(LogFileUtils.offsetFromFileName(firstLogFile.get()), partitionFiles); } - private List getTopicPartitionFiles(TopicPartition topicPartition) { - File[] files = brokerStorageDirectory.listFiles((dir, name) -> name.equals(topicPartition.toString())); + public boolean dirContainsTopicPartition(TopicPartition topicPartition, File logDir) { + File[] files = getTopicPartitionFiles(topicPartition, Collections.singleton(logDir)); + return files != null && files.length > 0; + } + + private File[] getTopicPartitionFiles(TopicPartition topicPartition) { + return getTopicPartitionFiles(topicPartition, brokerStorageDirectories); + } + + private File[] getTopicPartitionFiles(TopicPartition topicPartition, Set logDirs) { + File[] files = null; + for (File brokerDir : logDirs) { + files = brokerDir.listFiles((dir, name) -> name.equals(topicPartition.toString())); + // currently, we only expect one topic partition in one log dir + if (files != null && files.length != 0) { + break; + } + } + return files; + } + + private List getTopicPartitionFileNames(TopicPartition topicPartition) { + File[] files = getTopicPartitionFiles(topicPartition); if (files == null || files.length == 0) { throw new IllegalArgumentException(String.format("[BrokerId=%d] Directory for the topic-partition %s " + "was not found", brokerId, topicPartition)); diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/utils/TieredStorageTestUtils.java b/storage/src/test/java/org/apache/kafka/tiered/storage/utils/TieredStorageTestUtils.java index 1c2aef8a02c66..4db4ece844b99 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/utils/TieredStorageTestUtils.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/utils/TieredStorageTestUtils.java @@ -16,10 +16,12 @@ */ package org.apache.kafka.tiered.storage.utils; +import kafka.utils.TestUtils; import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.record.Record; +import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager; import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig; import org.apache.kafka.server.log.remote.storage.LocalTieredStorage; @@ -50,6 +52,7 @@ import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP; import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CONFIG_PREFIX_PROP; import static org.apache.kafka.server.config.ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG; +import static org.apache.kafka.storage.internals.log.CleanerConfig.LOG_CLEANER_ENABLE_PROP; public class TieredStorageTestUtils { @@ -149,6 +152,11 @@ public static Properties createPropsForRemoteStorage(String testClassName, overridingProps.setProperty(storageConfigPrefix(testClassName, DELETE_ON_CLOSE_CONFIG), "false"); // Set a small number of retry interval for retrying RemoteLogMetadataManager resources initialization to speed up the test overridingProps.setProperty(metadataConfigPrefix(testClassName, REMOTE_LOG_METADATA_INITIALIZATION_RETRY_INTERVAL_MS_PROP), RLMM_INIT_RETRY_INTERVAL_MS.toString()); + // Set 2 log dirs to make sure JBOD feature is working correctly + overridingProps.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, TestUtils.tempDir().getAbsolutePath() + "," + TestUtils.tempDir().getAbsolutePath()); + // Disable unnecessary log cleaner + overridingProps.setProperty(LOG_CLEANER_ENABLE_PROP, "false"); + return overridingProps; } diff --git a/streams/quickstart/java/pom.xml b/streams/quickstart/java/pom.xml index ea3b637fcbd4c..3cf31f1288367 100644 --- a/streams/quickstart/java/pom.xml +++ b/streams/quickstart/java/pom.xml @@ -26,7 +26,7 @@ org.apache.kafka streams-quickstart - 3.8.0-SNAPSHOT + 3.9.0-SNAPSHOT .. diff --git a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml index 89ba3dbab004f..f6034a115ec93 100644 --- a/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml +++ b/streams/quickstart/java/src/main/resources/archetype-resources/pom.xml @@ -29,7 +29,7 @@ UTF-8 - 3.8.0-SNAPSHOT + 3.9.0-SNAPSHOT 1.7.36 diff --git a/streams/quickstart/pom.xml b/streams/quickstart/pom.xml index b3164fd3db3e4..46cdea42fc406 100644 --- a/streams/quickstart/pom.xml +++ b/streams/quickstart/pom.xml @@ -22,7 +22,7 @@ org.apache.kafka streams-quickstart pom - 3.8.0-SNAPSHOT + 3.9.0-SNAPSHOT Kafka Streams :: Quickstart diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 70e4cba9625ac..25e86928b00c6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -45,6 +45,7 @@ import org.apache.kafka.streams.internals.UpgradeFromValues; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.streams.processor.assignment.TaskAssignor; import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier; import org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor; import org.apache.kafka.streams.processor.internals.assignment.RackAwareTaskAssignor; @@ -820,6 +821,10 @@ public class StreamsConfig extends AbstractConfig { + "optimization algorithm favors minimizing cross rack traffic or minimize the movement of tasks in existing assignment. If set a larger value " + RackAwareTaskAssignor.class.getName() + " will " + "optimize to maintain the existing assignment. The default value is null which means it will use default non_overlap cost values in different assignors."; + @SuppressWarnings("WeakerAccess") + public static final String TASK_ASSIGNOR_CLASS_CONFIG = "task.assignor.class"; + private static final String TASK_ASSIGNOR_CLASS_DOC = "A task assignor class or class name implementing the " + + TaskAssignor.class.getName() + " interface. Defaults to the HighAvailabilityTaskAssignor class."; /** * {@code topology.optimization} @@ -980,6 +985,11 @@ public class StreamsConfig extends AbstractConfig { null, Importance.MEDIUM, RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_DOC) + .define(TASK_ASSIGNOR_CLASS_CONFIG, + Type.STRING, + null, + Importance.MEDIUM, + TASK_ASSIGNOR_CLASS_DOC) .define(REPLICATION_FACTOR_CONFIG, Type.INT, -1, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java index 6b70d2702c2bc..394c130058801 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java @@ -172,8 +172,7 @@ public KStream join(final KStream lhs, final TimeTrackerSupplier sharedTimeTrackerSupplier = new TimeTrackerSupplier(); final JoinWindowsInternal internalWindows = new JoinWindowsInternal(windows); - final KStreamKStreamJoin joinThis = new KStreamKStreamJoin<>( - true, + final KStreamKStreamJoinLeftSide joinThis = new KStreamKStreamJoinLeftSide<>( otherWindowStore.name(), internalWindows, joiner, @@ -182,8 +181,7 @@ public KStream join(final KStream lhs, sharedTimeTrackerSupplier ); - final KStreamKStreamJoin joinOther = new KStreamKStreamJoin<>( - false, + final KStreamKStreamJoinRightSide joinOther = new KStreamKStreamJoinRightSide<>( thisWindowStore.name(), internalWindows, AbstractStream.reverseJoinerWithKey(joiner), diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index b8b48ff2c4df6..5646802aa9492 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -23,7 +23,6 @@ import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTracker; import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTrackerSupplier; import org.apache.kafka.streams.processor.api.ContextualProcessor; -import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; @@ -43,7 +42,7 @@ import static org.apache.kafka.streams.StreamsConfig.InternalConfig.EMIT_INTERVAL_MS_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX; import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor; -class KStreamKStreamJoin implements ProcessorSupplier { +abstract class KStreamKStreamJoin implements ProcessorSupplier { private static final Logger LOG = LoggerFactory.getLogger(KStreamKStreamJoin.class); private final String otherWindowName; @@ -55,28 +54,22 @@ class KStreamKStreamJoin implements ProcessorSupplier outerJoinWindowName; - private final ValueJoinerWithKey joiner; + private final ValueJoinerWithKey joiner; private final TimeTrackerSupplier sharedTimeTrackerSupplier; - KStreamKStreamJoin(final boolean isLeftSide, - final String otherWindowName, + KStreamKStreamJoin(final String otherWindowName, final JoinWindowsInternal windows, - final ValueJoinerWithKey joiner, + final ValueJoinerWithKey joiner, final boolean outer, final Optional outerJoinWindowName, + final long joinBeforeMs, + final long joinAfterMs, final TimeTrackerSupplier sharedTimeTrackerSupplier) { - this.isLeftSide = isLeftSide; this.otherWindowName = otherWindowName; - if (isLeftSide) { - this.joinBeforeMs = windows.beforeMs; - this.joinAfterMs = windows.afterMs; - } else { - this.joinBeforeMs = windows.afterMs; - this.joinAfterMs = windows.beforeMs; - } + this.joinBeforeMs = joinBeforeMs; + this.joinAfterMs = joinAfterMs; this.windowsAfterMs = windows.afterMs; this.windowsBeforeMs = windows.beforeMs; this.joinGraceMs = windows.gracePeriodMs(); @@ -87,15 +80,10 @@ class KStreamKStreamJoin implements ProcessorSupplier get() { - return new KStreamKStreamJoinProcessor(); - } - - private class KStreamKStreamJoinProcessor extends ContextualProcessor { - private WindowStore otherWindowStore; + protected abstract class KStreamKStreamJoinProcessor extends ContextualProcessor { + private WindowStore otherWindowStore; private Sensor droppedRecordsSensor; - private Optional, LeftOrRightValue>> outerJoinStore = Optional.empty(); + private Optional, LeftOrRightValue>> outerJoinStore = Optional.empty(); private InternalProcessorContext internalProcessorContext; private TimeTracker sharedTimeTracker; @@ -122,13 +110,10 @@ public void init(final ProcessorContext context) { } } - @SuppressWarnings("unchecked") @Override - public void process(final Record record) { + public void process(final Record record) { final long inputRecordTimestamp = record.timestamp(); - final long timeFrom = Math.max(0L, inputRecordTimestamp - joinBeforeMs); - final long timeTo = Math.max(0L, inputRecordTimestamp + joinAfterMs); sharedTimeTracker.advanceStreamTime(inputRecordTimestamp); @@ -144,26 +129,11 @@ public void process(final Record record) { outerJoinStore.ifPresent(store -> emitNonJoinedOuterRecords(store, record)); } - boolean needOuterJoin = outer; - try (final WindowStoreIterator iter = otherWindowStore.fetch(record.key(), timeFrom, timeTo)) { - while (iter.hasNext()) { - needOuterJoin = false; - final KeyValue otherRecord = iter.next(); - final long otherRecordTimestamp = otherRecord.key; - - outerJoinStore.ifPresent(store -> { - // use putIfAbsent to first read and see if there's any values for the key, - // if yes delete the key, otherwise do not issue a put; - // we may delete some values with the same key early but since we are going - // range over all values of the same key even after failure, since the other window-store - // is only cleaned up by stream time, so this is okay for at-least-once. - store.putIfAbsent(TimestampedKeyAndJoinSide.make(!isLeftSide, record.key(), otherRecordTimestamp), null); - }); - - context().forward( - record.withValue(joiner.apply(record.key(), record.value(), otherRecord.value)) - .withTimestamp(Math.max(inputRecordTimestamp, otherRecordTimestamp))); - } + final long timeFrom = Math.max(0L, inputRecordTimestamp - joinBeforeMs); + final long timeTo = Math.max(0L, inputRecordTimestamp + joinAfterMs); + try (final WindowStoreIterator iter = otherWindowStore.fetch(record.key(), timeFrom, timeTo)) { + final boolean needOuterJoin = outer && !iter.hasNext(); + iter.forEachRemaining(otherRecord -> emitInnerJoin(record, otherRecord, inputRecordTimestamp)); if (needOuterJoin) { // The maxStreamTime contains the max time observed in both sides of the join. @@ -187,17 +157,24 @@ public void process(final Record record) { context().forward(record.withValue(joiner.apply(record.key(), record.value(), null))); } else { sharedTimeTracker.updatedMinTime(inputRecordTimestamp); - outerJoinStore.ifPresent(store -> store.put( - TimestampedKeyAndJoinSide.make(isLeftSide, record.key(), inputRecordTimestamp), - LeftOrRightValue.make(isLeftSide, record.value()))); + putInOuterJoinStore(record); } } } } - private void emitNonJoinedOuterRecords( - final KeyValueStore, LeftOrRightValue> store, - final Record record) { + protected abstract TimestampedKeyAndJoinSide makeThisKey(final K key, final long inputRecordTimestamp); + + protected abstract LeftOrRightValue makeThisValue(final VThis thisValue); + + protected abstract TimestampedKeyAndJoinSide makeOtherKey(final K key, final long timestamp); + + protected abstract VThis getThisValue(final LeftOrRightValue leftOrRightValue); + + protected abstract VOther getOtherValue(final LeftOrRightValue leftOrRightValue); + + private void emitNonJoinedOuterRecords(final KeyValueStore, LeftOrRightValue> store, + final Record record) { // calling `store.all()` creates an iterator what is an expensive operation on RocksDB; // to reduce runtime cost, we try to avoid paying those cost @@ -221,26 +198,24 @@ private void emitNonJoinedOuterRecords( // reset to MAX_VALUE in case the store is empty sharedTimeTracker.minTime = Long.MAX_VALUE; - try (final KeyValueIterator, LeftOrRightValue> it = store.all()) { + try (final KeyValueIterator, LeftOrRightValue> it = store.all()) { TimestampedKeyAndJoinSide prevKey = null; boolean outerJoinLeftWindowOpen = false; boolean outerJoinRightWindowOpen = false; while (it.hasNext()) { + final KeyValue, LeftOrRightValue> nextKeyValue = it.next(); + final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide = nextKeyValue.key; + sharedTimeTracker.minTime = timestampedKeyAndJoinSide.getTimestamp(); if (outerJoinLeftWindowOpen && outerJoinRightWindowOpen) { // if windows are open for both joinSides we can break since there are no more candidates to emit break; } - final KeyValue, LeftOrRightValue> next = it.next(); - final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide = next.key; - final long timestamp = timestampedKeyAndJoinSide.getTimestamp(); - sharedTimeTracker.minTime = timestamp; // Continue with the next outer record if window for this joinSide has not closed yet // There might be an outer record for the other joinSide which window has not closed yet // We rely on the ordering of KeyValueIterator - final long outerJoinLookBackTimeMs = getOuterJoinLookBackTimeMs(timestampedKeyAndJoinSide); - if (sharedTimeTracker.minTime + outerJoinLookBackTimeMs + joinGraceMs >= sharedTimeTracker.streamTime) { + if (isOuterJoinWindowOpen(timestampedKeyAndJoinSide)) { if (timestampedKeyAndJoinSide.isLeftSide()) { outerJoinLeftWindowOpen = true; // there are no more candidates to emit on left-outerJoin-side } else { @@ -249,13 +224,9 @@ private void emitNonJoinedOuterRecords( // We continue with the next outer record continue; } - - final K key = timestampedKeyAndJoinSide.getKey(); - final LeftOrRightValue leftOrRightValue = next.value; - final VOut nullJoinedValue = getNullJoinedValue(key, leftOrRightValue); - context().forward( - record.withKey(key).withValue(nullJoinedValue).withTimestamp(timestamp) - ); + + final LeftOrRightValue leftOrRightValue = nextKeyValue.value; + forwardNonJoinedOuterRecords(record, timestampedKeyAndJoinSide, leftOrRightValue); if (prevKey != null && !prevKey.equals(timestampedKeyAndJoinSide)) { // blind-delete the previous key from the outer window store now it is emitted; @@ -275,20 +246,22 @@ private void emitNonJoinedOuterRecords( } } - @SuppressWarnings("unchecked") - private VOut getNullJoinedValue( - final K key, - final LeftOrRightValue leftOrRightValue) { - // depending on the JoinSide fill in the joiner key and joiner values - if (isLeftSide) { - return joiner.apply(key, - leftOrRightValue.getLeftValue(), - leftOrRightValue.getRightValue()); - } else { - return joiner.apply(key, - (V1) leftOrRightValue.getRightValue(), - (V2) leftOrRightValue.getLeftValue()); - } + private void forwardNonJoinedOuterRecords(final Record record, + final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide, + final LeftOrRightValue leftOrRightValue) { + final K key = timestampedKeyAndJoinSide.getKey(); + final long timestamp = timestampedKeyAndJoinSide.getTimestamp(); + final VThis thisValue = getThisValue(leftOrRightValue); + final VOther otherValue = getOtherValue(leftOrRightValue); + final VOut nullJoinedValue = joiner.apply(key, thisValue, otherValue); + context().forward( + record.withKey(key).withValue(nullJoinedValue).withTimestamp(timestamp) + ); + } + + private boolean isOuterJoinWindowOpen(final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide) { + final long outerJoinLookBackTimeMs = getOuterJoinLookBackTimeMs(timestampedKeyAndJoinSide); + return sharedTimeTracker.minTime + outerJoinLookBackTimeMs + joinGraceMs >= sharedTimeTracker.streamTime; } private long getOuterJoinLookBackTimeMs( @@ -301,6 +274,31 @@ private long getOuterJoinLookBackTimeMs( } } + private void emitInnerJoin(final Record thisRecord, final KeyValue otherRecord, + final long inputRecordTimestamp) { + outerJoinStore.ifPresent(store -> { + // use putIfAbsent to first read and see if there's any values for the key, + // if yes delete the key, otherwise do not issue a put; + // we may delete some values with the same key early but since we are going + // range over all values of the same key even after failure, since the other window-store + // is only cleaned up by stream time, so this is okay for at-least-once. + final TimestampedKeyAndJoinSide otherKey = makeOtherKey(thisRecord.key(), otherRecord.key); + store.putIfAbsent(otherKey, null); + }); + + context().forward( + thisRecord.withValue(joiner.apply(thisRecord.key(), thisRecord.value(), otherRecord.value)) + .withTimestamp(Math.max(inputRecordTimestamp, otherRecord.key))); + } + + private void putInOuterJoinStore(final Record thisRecord) { + outerJoinStore.ifPresent(store -> { + final TimestampedKeyAndJoinSide thisKey = makeThisKey(thisRecord.key(), thisRecord.timestamp()); + final LeftOrRightValue thisValue = makeThisValue(thisRecord.value()); + store.put(thisKey, thisValue); + }); + } + @Override public void close() { sharedTimeTrackerSupplier.remove(context().taskId()); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinLeftSide.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinLeftSide.java new file mode 100644 index 0000000000000..3b4ee5b33930a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinLeftSide.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.streams.kstream.ValueJoinerWithKey; +import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTrackerSupplier; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.state.internals.LeftOrRightValue; +import org.apache.kafka.streams.state.internals.TimestampedKeyAndJoinSide; + +import java.util.Optional; + +class KStreamKStreamJoinLeftSide extends KStreamKStreamJoin { + + KStreamKStreamJoinLeftSide(final String otherWindowName, + final JoinWindowsInternal windows, + final ValueJoinerWithKey joiner, + final boolean outer, + final Optional outerJoinWindowName, + final TimeTrackerSupplier sharedTimeTrackerSupplier) { + super(otherWindowName, windows, joiner, outer, outerJoinWindowName, windows.beforeMs, windows.afterMs, + sharedTimeTrackerSupplier); + } + + @Override + public Processor get() { + return new KStreamKStreamJoinLeftProcessor(); + } + + private class KStreamKStreamJoinLeftProcessor extends KStreamKStreamJoinProcessor { + + @Override + public TimestampedKeyAndJoinSide makeThisKey(final K key, final long timestamp) { + return TimestampedKeyAndJoinSide.makeLeft(key, timestamp); + } + + @Override + public LeftOrRightValue makeThisValue(final VLeft thisValue) { + return LeftOrRightValue.makeLeftValue(thisValue); + } + + @Override + public TimestampedKeyAndJoinSide makeOtherKey(final K key, final long timestamp) { + return TimestampedKeyAndJoinSide.makeRight(key, timestamp); + } + + @Override + public VLeft getThisValue(final LeftOrRightValue leftOrRightValue) { + return leftOrRightValue.getLeftValue(); + } + + @Override + public VRight getOtherValue(final LeftOrRightValue leftOrRightValue) { + return leftOrRightValue.getRightValue(); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinRightSide.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinRightSide.java new file mode 100644 index 0000000000000..e4bcfe4e10517 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinRightSide.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.streams.kstream.ValueJoinerWithKey; +import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTrackerSupplier; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.state.internals.LeftOrRightValue; +import org.apache.kafka.streams.state.internals.TimestampedKeyAndJoinSide; + +import java.util.Optional; + +class KStreamKStreamJoinRightSide extends KStreamKStreamJoin { + + KStreamKStreamJoinRightSide(final String otherWindowName, + final JoinWindowsInternal windows, + final ValueJoinerWithKey joiner, + final boolean outer, + final Optional outerJoinWindowName, + final TimeTrackerSupplier sharedTimeTrackerSupplier) { + super(otherWindowName, windows, joiner, outer, outerJoinWindowName, windows.afterMs, windows.beforeMs, + sharedTimeTrackerSupplier); + } + + @Override + public Processor get() { + return new KStreamKStreamRightJoinProcessor(); + } + + private class KStreamKStreamRightJoinProcessor extends KStreamKStreamJoinProcessor { + @Override + public TimestampedKeyAndJoinSide makeThisKey(final K key, final long timestamp) { + return TimestampedKeyAndJoinSide.makeRight(key, timestamp); + } + + @Override + public LeftOrRightValue makeThisValue(final VRight thisValue) { + return LeftOrRightValue.makeRightValue(thisValue); + } + + @Override + public TimestampedKeyAndJoinSide makeOtherKey(final K key, final long timestamp) { + return TimestampedKeyAndJoinSide.makeLeft(key, timestamp); + } + + @Override + public VRight getThisValue(final LeftOrRightValue leftOrRightValue) { + return leftOrRightValue.getRightValue(); + } + + @Override + public VLeft getOtherValue(final LeftOrRightValue leftOrRightValue) { + return leftOrRightValue.getLeftValue(); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/ApplicationState.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/ApplicationState.java index 0fb4f06f205a2..7c714e111b49b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/ApplicationState.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/ApplicationState.java @@ -17,8 +17,8 @@ package org.apache.kafka.streams.processor.assignment; import java.util.Map; -import java.util.Set; import org.apache.kafka.streams.errors.TaskAssignmentException; +import org.apache.kafka.streams.processor.TaskId; /** * A read-only metadata class representing the state of the application and the current rebalance. @@ -48,5 +48,5 @@ public interface ApplicationState { /** * @return the set of all tasks in this topology which must be assigned */ - Set allTasks(); + Map allTasks(); } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/KafkaStreamsAssignment.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/KafkaStreamsAssignment.java index bd2497e261d89..f5205c8422bdf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/KafkaStreamsAssignment.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/KafkaStreamsAssignment.java @@ -16,11 +16,14 @@ */ package org.apache.kafka.streams.processor.assignment; -import static java.util.Collections.unmodifiableSet; +import static java.util.Collections.unmodifiableMap; import java.time.Instant; +import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; import org.apache.kafka.streams.processor.TaskId; /** @@ -30,7 +33,7 @@ public class KafkaStreamsAssignment { private final ProcessId processId; - private final Set assignment; + private final Map tasks; private final Optional followupRebalanceDeadline; /** @@ -44,7 +47,8 @@ public class KafkaStreamsAssignment { * @return a new KafkaStreamsAssignment object with the given processId and assignment */ public static KafkaStreamsAssignment of(final ProcessId processId, final Set assignment) { - return new KafkaStreamsAssignment(processId, assignment, Optional.empty()); + final Map tasks = assignment.stream().collect(Collectors.toMap(AssignedTask::id, Function.identity())); + return new KafkaStreamsAssignment(processId, tasks, Optional.empty()); } /** @@ -61,14 +65,14 @@ public static KafkaStreamsAssignment of(final ProcessId processId, final Set assignment, + final Map tasks, final Optional followupRebalanceDeadline) { this.processId = processId; - this.assignment = unmodifiableSet(assignment); + this.tasks = tasks; this.followupRebalanceDeadline = followupRebalanceDeadline; } @@ -82,10 +86,18 @@ public ProcessId processId() { /** * - * @return a set of assigned tasks that are part of this {@code KafkaStreamsAssignment} + * @return a read-only set of assigned tasks that are part of this {@code KafkaStreamsAssignment} */ - public Set assignment() { - return assignment; + public Map tasks() { + return unmodifiableMap(tasks); + } + + public void assignTask(final AssignedTask newTask) { + tasks.put(newTask.id(), newTask); + } + + public void removeTask(final AssignedTask removedTask) { + tasks.remove(removedTask.id()); } /** @@ -125,5 +137,25 @@ public TaskId id() { public Type type() { return taskType; } + + @Override + public int hashCode() { + final int prime = 31; + int result = prime + this.id.hashCode(); + result = prime * result + this.type().hashCode(); + return result; + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + final AssignedTask other = (AssignedTask) obj; + return this.id.equals(other.id()) && this.taskType == other.taskType; + } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/ProcessId.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/ProcessId.java index 0b09aae1f8d67..c8cf322810502 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/ProcessId.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/ProcessId.java @@ -39,4 +39,21 @@ public UUID id() { public String toString() { return "ProcessId{id=" + id + "}"; } + + @Override + public int hashCode() { + return id.hashCode(); + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + final ProcessId other = (ProcessId) obj; + return this.id.equals(other.id()); + } } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignmentUtils.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignmentUtils.java index 3945f95dc7284..d2e30d538050b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignmentUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignmentUtils.java @@ -16,14 +16,70 @@ */ package org.apache.kafka.streams.processor.assignment; +import static org.apache.kafka.streams.processor.internals.assignment.RackAwareTaskAssignor.STANDBY_OPTIMIZER_MAX_ITERATION; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.Set; import java.util.SortedSet; +import java.util.UUID; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.assignment.KafkaStreamsAssignment.AssignedTask; +import org.apache.kafka.streams.processor.internals.assignment.ConstrainedPrioritySet; +import org.apache.kafka.streams.processor.internals.assignment.Graph; +import org.apache.kafka.streams.processor.internals.assignment.MinTrafficGraphConstructor; +import org.apache.kafka.streams.processor.internals.assignment.RackAwareGraphConstructor; +import org.apache.kafka.streams.processor.internals.assignment.RackAwareGraphConstructorFactory; +import org.apache.kafka.streams.StreamsConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A set of utilities to help implement task assignment via the {@link TaskAssignor} */ public final class TaskAssignmentUtils { + private static final Logger LOG = LoggerFactory.getLogger(TaskAssignmentUtils.class); + + private TaskAssignmentUtils() {} + + /** + * Return a "no-op" assignment that just copies the previous assignment of tasks to KafkaStreams clients + * + * @param applicationState the metadata and other info describing the current application state + * + * @return a new map containing an assignment that replicates exactly the previous assignment reported + * in the applicationState + */ + public static Map identityAssignment(final ApplicationState applicationState) { + final Map assignments = new HashMap<>(); + applicationState.kafkaStreamsStates(false).forEach((processId, state) -> { + final Set tasks = new HashSet<>(); + state.previousActiveTasks().forEach(taskId -> { + tasks.add(new AssignedTask(taskId, + AssignedTask.Type.ACTIVE)); + }); + state.previousStandbyTasks().forEach(taskId -> { + tasks.add(new AssignedTask(taskId, + AssignedTask.Type.STANDBY)); + }); + + final KafkaStreamsAssignment newAssignment = KafkaStreamsAssignment.of(processId, tasks); + assignments.put(processId, newAssignment); + }); + return assignments; + } + /** * Assign standby tasks to KafkaStreams clients according to the default logic. *

      @@ -32,62 +88,764 @@ public final class TaskAssignmentUtils { * @param applicationState the metadata and other info describing the current application state * @param kafkaStreamsAssignments the current assignment of tasks to KafkaStreams clients * - * @return a new map containing the mappings from KafkaStreamsAssignments updated with the default - * standby assignment + * @return a new map containing the mappings from KafkaStreamsAssignments updated with the default standby assignment */ - public static Map defaultStandbyTaskAssignment( - final ApplicationState applicationState, - final Map kafkaStreamsAssignments - ) { - throw new UnsupportedOperationException("Not Implemented."); + public static Map defaultStandbyTaskAssignment(final ApplicationState applicationState, + final Map kafkaStreamsAssignments) { + if (!applicationState.assignmentConfigs().rackAwareAssignmentTags().isEmpty()) { + return tagBasedStandbyTaskAssignment(applicationState, kafkaStreamsAssignments); + } else if (canPerformRackAwareOptimization(applicationState, AssignedTask.Type.STANDBY)) { + return tagBasedStandbyTaskAssignment(applicationState, kafkaStreamsAssignments); + } else { + return loadBasedStandbyTaskAssignment(applicationState, kafkaStreamsAssignments); + } } /** - * Optimize the active task assignment for rack-awareness + * Optimize active task assignment for rack awareness. This optimization is based on the + * {@link StreamsConfig#RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG trafficCost} + * and {@link StreamsConfig#RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG nonOverlapCost} + * configs which balance cross rack traffic minimization and task movement. + * Setting {@code trafficCost} to a larger number reduces the overall cross rack traffic of the resulting + * assignment, but can increase the number of tasks shuffled around between clients. + * Setting {@code nonOverlapCost} to a larger number increases the affinity of tasks to their intended client + * and reduces the amount by which the rack-aware optimization can shuffle tasks around, at the cost of higher + * cross-rack traffic. + * In an extreme case, if we set {@code nonOverlapCost} to 0 and @{code trafficCost} to a positive value, + * the resulting assignment will have an absolute minimum of cross rack traffic. If we set {@code trafficCost} to 0, + * and {@code nonOverlapCost} to a positive value, the resulting assignment will be identical to the input assignment. + *

      + * Note: this method will modify the input {@link KafkaStreamsAssignment} objects and return the same map. + * It does not make a copy of the map or the KafkaStreamsAssignment objects. + *

      + * This method optimizes cross-rack traffic for active tasks only. For standby task optimization, + * use {@link #optimizeRackAwareStandbyTasks}. * * @param applicationState the metadata and other info describing the current application state * @param kafkaStreamsAssignments the current assignment of tasks to KafkaStreams clients - * @param tasks the set of tasks to reassign if possible. Must already be assigned - * to a KafkaStreams client + * @param tasks the set of tasks to reassign if possible. Must already be assigned to a KafkaStreams client * - * @return a new map containing the mappings from KafkaStreamsAssignments updated with the default - * rack-aware assignment for active tasks + * @return a map with the KafkaStreamsAssignments updated to minimize cross-rack traffic for active tasks */ - public static Map optimizeRackAwareActiveTasks( - final ApplicationState applicationState, - final Map kafkaStreamsAssignments, - final SortedSet tasks - ) { - throw new UnsupportedOperationException("Not Implemented."); + public static Map optimizeRackAwareActiveTasks(final ApplicationState applicationState, + final Map kafkaStreamsAssignments, + final SortedSet tasks) { + if (tasks.isEmpty()) { + return kafkaStreamsAssignments; + } + + if (!canPerformRackAwareOptimization(applicationState, AssignedTask.Type.ACTIVE)) { + return kafkaStreamsAssignments; + } + + final int crossRackTrafficCost = applicationState.assignmentConfigs().rackAwareTrafficCost(); + final int nonOverlapCost = applicationState.assignmentConfigs().rackAwareNonOverlapCost(); + + final Map kafkaStreamsStates = applicationState.kafkaStreamsStates(false); + final List taskIds = new ArrayList<>(tasks); + + final Map> topicPartitionsByTaskId = applicationState.allTasks().values().stream() + .filter(taskInfo -> tasks.contains(taskInfo.id())) + .collect(Collectors.toMap(TaskInfo::id, TaskInfo::topicPartitions)); + + final Map> clientRacks = new HashMap<>(); + final List clientIds = new ArrayList<>(); + final Map assignmentsByUuid = new HashMap<>(); + + for (final Map.Entry entry : kafkaStreamsAssignments.entrySet()) { + final UUID uuid = entry.getKey().id(); + clientIds.add(uuid); + clientRacks.put(uuid, kafkaStreamsStates.get(entry.getKey()).rackId()); + } + + final long initialCost = computeTotalAssignmentCost( + topicPartitionsByTaskId, + taskIds, + clientIds, + assignmentsByUuid, + clientRacks, + crossRackTrafficCost, + nonOverlapCost, + false, + false + ); + + LOG.info("Assignment before active task optimization has cost {}", initialCost); + + final RackAwareGraphConstructor graphConstructor = RackAwareGraphConstructorFactory.create( + applicationState.assignmentConfigs().rackAwareAssignmentStrategy(), taskIds); + + final AssignmentGraph assignmentGraph = buildTaskGraph( + assignmentsByUuid, + clientRacks, + taskIds, + clientIds, + topicPartitionsByTaskId, + crossRackTrafficCost, + nonOverlapCost, + false, + false, + graphConstructor + ); + + assignmentGraph.graph.solveMinCostFlow(); + + graphConstructor.assignTaskFromMinCostFlow( + assignmentGraph.graph, + clientIds, + taskIds, + assignmentsByUuid, + assignmentGraph.taskCountByClient, + assignmentGraph.clientByTask, + (assignment, taskId) -> assignment.assignTask(new AssignedTask(taskId, AssignedTask.Type.ACTIVE)), + (assignment, taskId) -> assignment.removeTask(new AssignedTask(taskId, AssignedTask.Type.ACTIVE)), + (assignment, taskId) -> assignment.tasks().containsKey(taskId) + ); + + return kafkaStreamsAssignments; } /** - * Optimize the standby task assignment for rack-awareness + * Optimize standby task assignment for rack awareness. This optimization is based on the + * {@link StreamsConfig#RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG trafficCost} + * and {@link StreamsConfig#RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG nonOverlapCost} + * configs which balance cross rack traffic minimization and task movement. + * Setting {@code trafficCost} to a larger number reduces the overall cross rack traffic of the resulting + * assignment, but can increase the number of tasks shuffled around between clients. + * Setting {@code nonOverlapCost} to a larger number increases the affinity of tasks to their intended client + * and reduces the amount by which the rack-aware optimization can shuffle tasks around, at the cost of higher + * cross-rack traffic. + * In an extreme case, if we set {@code nonOverlapCost} to 0 and @{code trafficCost} to a positive value, + * the resulting assignment will have an absolute minimum of cross rack traffic. If we set {@code trafficCost} to 0, + * and {@code nonOverlapCost} to a positive value, the resulting assignment will be identical to the input assignment. + *

      + * Note: this method will modify the input {@link KafkaStreamsAssignment} objects and return the same map. + * It does not make a copy of the map or the KafkaStreamsAssignment objects. + *

      + * This method optimizes cross-rack traffic for standby tasks only. For active task optimization, + * use {@link #optimizeRackAwareActiveTasks}. * * @param kafkaStreamsAssignments the current assignment of tasks to KafkaStreams clients * @param applicationState the metadata and other info describing the current application state * - * @return a new map containing the mappings from KafkaStreamsAssignments updated with the default - * rack-aware assignment for standby tasks + * @return a map with the KafkaStreamsAssignments updated to minimize cross-rack traffic for standby tasks */ - public static Map optimizeRackAwareStandbyTasks( - final ApplicationState applicationState, - final Map kafkaStreamsAssignments - ) { - throw new UnsupportedOperationException("Not Implemented."); + public static Map optimizeRackAwareStandbyTasks(final ApplicationState applicationState, + final Map kafkaStreamsAssignments) { + if (!canPerformRackAwareOptimization(applicationState, AssignedTask.Type.STANDBY)) { + return kafkaStreamsAssignments; + } + + final int crossRackTrafficCost = applicationState.assignmentConfigs().rackAwareTrafficCost(); + final int nonOverlapCost = applicationState.assignmentConfigs().rackAwareNonOverlapCost(); + + final Map> topicPartitionsByTaskId = + applicationState.allTasks().values().stream().collect(Collectors.toMap( + TaskInfo::id, + t -> t.topicPartitions().stream().filter(TaskTopicPartition::isChangelog).collect(Collectors.toSet())) + ); + final List taskIds = new ArrayList<>(topicPartitionsByTaskId.keySet()); + + final Map kafkaStreamsStates = applicationState.kafkaStreamsStates(false); + + final Map> clientRacks = new HashMap<>(); + final List clientIds = new ArrayList<>(); + final Map assignmentsByUuid = new HashMap<>(); + + for (final Map.Entry entry : kafkaStreamsAssignments.entrySet()) { + final UUID uuid = entry.getKey().id(); + clientIds.add(uuid); + clientRacks.put(uuid, kafkaStreamsStates.get(entry.getKey()).rackId()); + } + + final long initialCost = computeTotalAssignmentCost( + topicPartitionsByTaskId, + taskIds, + clientIds, + assignmentsByUuid, + clientRacks, + crossRackTrafficCost, + nonOverlapCost, + true, + true + ); + LOG.info("Assignment before standby task optimization has cost {}", initialCost); + + final MoveStandbyTaskPredicate moveablePredicate = getStandbyTaskMovePredicate(applicationState); + final BiFunction> getMovableTasks = (source, destination) -> { + return source.tasks().values().stream() + .filter(task -> task.type() == AssignedTask.Type.STANDBY) + .filter(task -> !destination.tasks().containsKey(task.id())) + .filter(task -> { + final KafkaStreamsState sourceState = kafkaStreamsStates.get(source.processId()); + final KafkaStreamsState destinationState = kafkaStreamsStates.get(source.processId()); + return moveablePredicate.canMoveStandbyTask(sourceState, destinationState, task.id(), kafkaStreamsAssignments); + }) + .map(AssignedTask::id) + .sorted() + .collect(Collectors.toList()); + }; + + final long startTime = System.currentTimeMillis(); + boolean taskMoved = true; + int round = 0; + final RackAwareGraphConstructor graphConstructor = RackAwareGraphConstructorFactory.create( + applicationState.assignmentConfigs().rackAwareAssignmentStrategy(), taskIds); + while (taskMoved && round < STANDBY_OPTIMIZER_MAX_ITERATION) { + taskMoved = false; + round++; + for (int i = 0; i < kafkaStreamsAssignments.size(); i++) { + final UUID clientId1 = clientIds.get(i); + final KafkaStreamsAssignment clientState1 = kafkaStreamsAssignments.get(new ProcessId(clientId1)); + for (int j = i + 1; j < kafkaStreamsAssignments.size(); j++) { + final UUID clientId2 = clientIds.get(i); + final KafkaStreamsAssignment clientState2 = kafkaStreamsAssignments.get(new ProcessId(clientId2)); + + final String rack1 = clientRacks.get(clientState1.processId().id()).get(); + final String rack2 = clientRacks.get(clientState2.processId().id()).get(); + // Cross rack traffic can not be reduced if racks are the same + if (rack1.equals(rack2)) { + continue; + } + + final List movable1 = getMovableTasks.apply(clientState1, clientState2); + final List movable2 = getMovableTasks.apply(clientState2, clientState1); + + // There's no needed to optimize if one is empty because the optimization + // can only swap tasks to keep the client's load balanced + if (movable1.isEmpty() || movable2.isEmpty()) { + continue; + } + + final List taskIdList = Stream.concat(movable1.stream(), movable2.stream()) + .sorted() + .collect(Collectors.toList()); + final List clients = Stream.of(clientId1, clientId2).sorted().collect(Collectors.toList()); + + final AssignmentGraph assignmentGraph = buildTaskGraph( + assignmentsByUuid, + clientRacks, + taskIdList, + clients, + topicPartitionsByTaskId, + crossRackTrafficCost, + nonOverlapCost, + false, + false, + graphConstructor + ); + assignmentGraph.graph.solveMinCostFlow(); + + taskMoved |= graphConstructor.assignTaskFromMinCostFlow( + assignmentGraph.graph, + clientIds, + taskIds, + assignmentsByUuid, + assignmentGraph.taskCountByClient, + assignmentGraph.clientByTask, + (assignment, taskId) -> assignment.assignTask(new AssignedTask(taskId, AssignedTask.Type.STANDBY)), + (assignment, taskId) -> assignment.removeTask(new AssignedTask(taskId, AssignedTask.Type.STANDBY)), + (assignment, taskId) -> assignment.tasks().containsKey(taskId) && assignment.tasks().get(taskId).type() == AssignedTask.Type.STANDBY + ); + } + } + } + final long finalCost = computeTotalAssignmentCost( + topicPartitionsByTaskId, + taskIds, + clientIds, + assignmentsByUuid, + clientRacks, + crossRackTrafficCost, + nonOverlapCost, + true, + true + ); + + final long duration = System.currentTimeMillis() - startTime; + LOG.info("Assignment after {} rounds and {} milliseconds for standby task optimization is {}\n with cost {}", + round, duration, kafkaStreamsAssignments, finalCost); + return kafkaStreamsAssignments; + } + + private static long computeTotalAssignmentCost(final Map> topicPartitionsByTaskId, + final List taskIds, + final List clientIds, + final Map assignmentsByUuid, + final Map> clientRacks, + final int crossRackTrafficCost, + final int nonOverlapCost, + final boolean hasReplica, + final boolean isStandby) { + if (taskIds.isEmpty()) { + return 0; + } + + final RackAwareGraphConstructor graphConstructor = new MinTrafficGraphConstructor<>(); + final AssignmentGraph assignmentGraph = buildTaskGraph( + assignmentsByUuid, + clientRacks, + taskIds, + clientIds, + topicPartitionsByTaskId, + crossRackTrafficCost, + nonOverlapCost, + hasReplica, + isStandby, + graphConstructor + ); + return assignmentGraph.graph.totalCost(); + } + + private static AssignmentGraph buildTaskGraph(final Map assignmentsByUuid, + final Map> clientRacks, + final List taskIds, + final List clientList, + final Map> topicPartitionsByTaskId, + final int crossRackTrafficCost, + final int nonOverlapCost, + final boolean hasReplica, + final boolean isStandby, + final RackAwareGraphConstructor graphConstructor) { + // Intentionally passed in empty -- these are actually outputs of the graph + final Map clientByTask = new HashMap<>(); + final Map taskCountByClient = new HashMap<>(); + + final Graph graph = graphConstructor.constructTaskGraph( + clientList, + taskIds, + assignmentsByUuid, + clientByTask, + taskCountByClient, + (assignment, taskId) -> assignment.tasks().containsKey(taskId), + (taskId, processId, inCurrentAssignment, unused0, unused1, unused2) -> { + final String clientRack = clientRacks.get(processId).get(); + final int assignmentChangeCost = !inCurrentAssignment ? nonOverlapCost : 0; + return assignmentChangeCost + getCrossRackTrafficCost(topicPartitionsByTaskId.get(taskId), clientRack, crossRackTrafficCost); + }, + crossRackTrafficCost, + nonOverlapCost, + hasReplica, + isStandby + ); + return new AssignmentGraph(graph, clientByTask, taskCountByClient); + } + + /** + * This internal structure is used to keep track of the graph solving outputs alongside the graph + * structure itself. + */ + private static final class AssignmentGraph { + public final Graph graph; + public final Map clientByTask; + public final Map taskCountByClient; + + public AssignmentGraph(final Graph graph, + final Map clientByTask, + final Map taskCountByClient) { + this.graph = graph; + this.clientByTask = clientByTask; + this.taskCountByClient = taskCountByClient; + } + } + + @FunctionalInterface + public interface MoveStandbyTaskPredicate { + boolean canMoveStandbyTask(final KafkaStreamsState source, + final KafkaStreamsState destination, + final TaskId taskId, + final Map kafkaStreamsAssignment); } /** - * Return a "no-op" assignment that just copies the previous assignment of tasks to KafkaStreams clients * - * @param applicationState the metadata and other info describing the current application state + * @return the traffic cost of assigning this {@param task} to the client {@param streamsState}. + */ + private static int getCrossRackTrafficCost(final Set topicPartitions, + final String clientRack, + final int crossRackTrafficCost) { + int cost = 0; + for (final TaskTopicPartition topicPartition : topicPartitions) { + final Optional> topicPartitionRacks = topicPartition.rackIds(); + if (!topicPartitionRacks.get().contains(clientRack)) { + cost += crossRackTrafficCost; + } + } + return cost; + } + + /** * - * @return a new map containing an assignment that replicates exactly the previous assignment reported - * in the applicationState + * @return whether the rack information is valid, and the {@code StreamsConfig#RACK_AWARE_ASSIGNMENT_STRATEGY_NONE} + * is set. */ - public static Map identityAssignment( - final ApplicationState applicationState - ) { - throw new UnsupportedOperationException("Not Implemented."); + private static boolean canPerformRackAwareOptimization(final ApplicationState applicationState, + final AssignedTask.Type taskType) { + final String rackAwareAssignmentStrategy = applicationState.assignmentConfigs().rackAwareAssignmentStrategy(); + if (StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_NONE.equals(rackAwareAssignmentStrategy)) { + return false; + } + return hasValidRackInformation(applicationState, taskType); + } + + /** + * This function returns whether the current application state has the required rack information + * to make assignment decisions with. + * + * @param taskType the type of task that we are trying to validate rack information for. + * + * @return whether rack-aware assignment decisions can be made for this application. + */ + private static boolean hasValidRackInformation(final ApplicationState applicationState, + final AssignedTask.Type taskType) { + for (final KafkaStreamsState state : applicationState.kafkaStreamsStates(false).values()) { + if (!hasValidRackInformation(state)) { + return false; + } + } + + for (final TaskInfo task : applicationState.allTasks().values()) { + if (!hasValidRackInformation(task, taskType)) { + return false; + } + } + return true; + } + + private static boolean hasValidRackInformation(final KafkaStreamsState state) { + if (!state.rackId().isPresent()) { + LOG.error("KafkaStreams client {} doesn't have a rack id configured.", state.processId().id()); + return false; + } + return true; + } + + private static boolean hasValidRackInformation(final TaskInfo task, + final AssignedTask.Type taskType) { + final Collection topicPartitions = taskType == AssignedTask.Type.STANDBY + ? task.topicPartitions().stream().filter(TaskTopicPartition::isChangelog).collect(Collectors.toSet()) + : task.topicPartitions(); + + for (final TaskTopicPartition topicPartition : topicPartitions) { + final Optional> racks = topicPartition.rackIds(); + if (!racks.isPresent() || racks.get().isEmpty()) { + LOG.error("Topic partition {} for task {} does not have racks configured.", topicPartition, task.id()); + return false; + } + } + return true; + } + + private static Map tagBasedStandbyTaskAssignment(final ApplicationState applicationState, + final Map kafkaStreamsAssignments) { + final int numStandbyReplicas = applicationState.assignmentConfigs().numStandbyReplicas(); + final Map streamStates = applicationState.kafkaStreamsStates(false); + + final Set rackAwareAssignmentTags = new HashSet<>(applicationState.assignmentConfigs().rackAwareAssignmentTags()); + final TagStatistics tagStatistics = new TagStatistics(applicationState); + + final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = standbyTaskPriorityListByLoad(streamStates, kafkaStreamsAssignments); + + final Set statefulTaskIds = applicationState.allTasks().values().stream() + .filter(TaskInfo::isStateful) + .map(TaskInfo::id) + .collect(Collectors.toSet()); + final Map tasksToRemainingStandbys = statefulTaskIds.stream() + .collect(Collectors.toMap(Function.identity(), t -> numStandbyReplicas)); + final Map clientsByUuid = kafkaStreamsAssignments.entrySet().stream().collect(Collectors.toMap( + entry -> entry.getKey().id(), + Map.Entry::getValue + )); + + final Map pendingStandbyTasksToClientId = new HashMap<>(); + for (final TaskId statefulTaskId : statefulTaskIds) { + for (final KafkaStreamsAssignment assignment : clientsByUuid.values()) { + if (assignment.tasks().containsKey(statefulTaskId)) { + assignStandbyTasksToClientsWithDifferentTags( + numStandbyReplicas, + standbyTaskClientsByTaskLoad, + statefulTaskId, + assignment.processId(), + rackAwareAssignmentTags, + streamStates, + kafkaStreamsAssignments, + tasksToRemainingStandbys, + tagStatistics.tagKeyToValues, + tagStatistics.tagEntryToClients, + pendingStandbyTasksToClientId + ); + } + } + } + + if (!tasksToRemainingStandbys.isEmpty()) { + assignPendingStandbyTasksToLeastLoadedClients(clientsByUuid, + numStandbyReplicas, + standbyTaskClientsByTaskLoad, + tasksToRemainingStandbys); + } + + return kafkaStreamsAssignments; + } + + private static Map loadBasedStandbyTaskAssignment(final ApplicationState applicationState, + final Map kafkaStreamsAssignments) { + final int numStandbyReplicas = applicationState.assignmentConfigs().numStandbyReplicas(); + final Map streamStates = applicationState.kafkaStreamsStates(false); + + final Set statefulTaskIds = applicationState.allTasks().values().stream() + .filter(TaskInfo::isStateful) + .map(TaskInfo::id) + .collect(Collectors.toSet()); + final Map tasksToRemainingStandbys = statefulTaskIds.stream() + .collect(Collectors.toMap(Function.identity(), t -> numStandbyReplicas)); + final Map clients = kafkaStreamsAssignments.entrySet().stream().collect(Collectors.toMap( + entry -> entry.getKey().id(), + Map.Entry::getValue + )); + + final ConstrainedPrioritySet standbyTaskClientsByTaskLoad = standbyTaskPriorityListByLoad(streamStates, kafkaStreamsAssignments); + standbyTaskClientsByTaskLoad.offerAll(streamStates.keySet().stream().map(ProcessId::id).collect(Collectors.toSet())); + for (final TaskId task : statefulTaskIds) { + assignStandbyTasksForActiveTask( + numStandbyReplicas, + clients, + tasksToRemainingStandbys, + standbyTaskClientsByTaskLoad, + task + ); + } + return kafkaStreamsAssignments; + } + + private static void assignStandbyTasksForActiveTask(final int numStandbyReplicas, + final Map clients, + final Map tasksToRemainingStandbys, + final ConstrainedPrioritySet standbyTaskClientsByTaskLoad, + final TaskId activeTaskId) { + int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId); + while (numRemainingStandbys > 0) { + final UUID client = standbyTaskClientsByTaskLoad.poll(activeTaskId); + if (client == null) { + break; + } + clients.get(client).assignTask(new AssignedTask(activeTaskId, AssignedTask.Type.STANDBY)); + numRemainingStandbys--; + standbyTaskClientsByTaskLoad.offer(client); + } + + tasksToRemainingStandbys.put(activeTaskId, numRemainingStandbys); + if (numRemainingStandbys > 0) { + LOG.warn("Unable to assign {} of {} standby tasks for task [{}]. " + + "There is not enough available capacity. You should " + + "increase the number of application instances " + + "to maintain the requested number of standby replicas.", + numRemainingStandbys, numStandbyReplicas, activeTaskId); + } + } + + private static void assignStandbyTasksToClientsWithDifferentTags(final int numberOfStandbyClients, + final ConstrainedPrioritySet standbyTaskClientsByTaskLoad, + final TaskId activeTaskId, + final ProcessId activeTaskClient, + final Set rackAwareAssignmentTags, + final Map clientStates, + final Map kafkaStreamsAssignments, + final Map tasksToRemainingStandbys, + final Map> tagKeyToValues, + final Map, Set> tagEntryToClients, + final Map pendingStandbyTasksToClientId) { + standbyTaskClientsByTaskLoad.offerAll(clientStates.keySet().stream() + .map(ProcessId::id).collect(Collectors.toSet())); + + // We set countOfUsedClients as 1 because client where active task is located has to be considered as used. + int countOfUsedClients = 1; + int numRemainingStandbys = tasksToRemainingStandbys.get(activeTaskId); + + final Map, Set> tagEntryToUsedClients = new HashMap<>(); + + ProcessId lastUsedClient = activeTaskClient; + do { + updateClientsOnAlreadyUsedTagEntries( + clientStates.get(lastUsedClient), + countOfUsedClients, + rackAwareAssignmentTags, + tagEntryToClients, + tagKeyToValues, + tagEntryToUsedClients + ); + + final UUID clientOnUnusedTagDimensions = standbyTaskClientsByTaskLoad.poll( + activeTaskId, uuid -> !isClientUsedOnAnyOfTheTagEntries(new ProcessId(uuid), tagEntryToUsedClients) + ); + + if (clientOnUnusedTagDimensions == null) { + break; + } + + final KafkaStreamsState clientStateOnUsedTagDimensions = clientStates.get(new ProcessId(clientOnUnusedTagDimensions)); + countOfUsedClients++; + numRemainingStandbys--; + + LOG.debug("Assigning {} out of {} standby tasks for an active task [{}] with client tags {}. " + + "Standby task client tags are {}.", + numberOfStandbyClients - numRemainingStandbys, numberOfStandbyClients, activeTaskId, + clientStates.get(activeTaskClient).clientTags(), + clientStateOnUsedTagDimensions.clientTags()); + + kafkaStreamsAssignments.get(clientStateOnUsedTagDimensions.processId()).assignTask( + new AssignedTask(activeTaskId, AssignedTask.Type.STANDBY) + ); + lastUsedClient = new ProcessId(clientOnUnusedTagDimensions); + } while (numRemainingStandbys > 0); + + if (numRemainingStandbys > 0) { + pendingStandbyTasksToClientId.put(activeTaskId, activeTaskClient); + tasksToRemainingStandbys.put(activeTaskId, numRemainingStandbys); + LOG.warn("Rack aware standby task assignment was not able to assign {} of {} standby tasks for the " + + "active task [{}] with the rack aware assignment tags {}. " + + "This may happen when there aren't enough application instances on different tag " + + "dimensions compared to an active and corresponding standby task. " + + "Consider launching application instances on different tag dimensions than [{}]. " + + "Standby task assignment will fall back to assigning standby tasks to the least loaded clients.", + numRemainingStandbys, numberOfStandbyClients, + activeTaskId, rackAwareAssignmentTags, + clientStates.get(activeTaskClient).clientTags()); + + } else { + tasksToRemainingStandbys.remove(activeTaskId); + } + } + + private static boolean isClientUsedOnAnyOfTheTagEntries(final ProcessId client, + final Map, Set> tagEntryToUsedClients) { + return tagEntryToUsedClients.values().stream().anyMatch(usedClients -> usedClients.contains(client)); + } + + private static void updateClientsOnAlreadyUsedTagEntries(final KafkaStreamsState usedClient, + final int countOfUsedClients, + final Set rackAwareAssignmentTags, + final Map, Set> tagEntryToClients, + final Map> tagKeyToValues, + final Map, Set> tagEntryToUsedClients) { + final Map usedClientTags = usedClient.clientTags(); + + for (final Map.Entry usedClientTagEntry : usedClientTags.entrySet()) { + final String tagKey = usedClientTagEntry.getKey(); + + if (!rackAwareAssignmentTags.contains(tagKey)) { + LOG.warn("Client tag with key [{}] will be ignored when computing rack aware standby " + + "task assignment because it is not part of the configured rack awareness [{}].", + tagKey, rackAwareAssignmentTags); + continue; + } + + final Set allTagValues = tagKeyToValues.get(tagKey); + + if (allTagValues.size() <= countOfUsedClients) { + allTagValues.forEach(tagValue -> tagEntryToUsedClients.remove(new KeyValue<>(tagKey, tagValue))); + } else { + final String tagValue = usedClientTagEntry.getValue(); + final KeyValue tagEntry = new KeyValue<>(tagKey, tagValue); + final Set clientsOnUsedTagValue = tagEntryToClients.get(tagEntry); + tagEntryToUsedClients.put(tagEntry, clientsOnUsedTagValue); + } + } + } + + private static MoveStandbyTaskPredicate getStandbyTaskMovePredicate(final ApplicationState applicationState) { + final boolean hasRackAwareAssignmentTags = !applicationState.assignmentConfigs().rackAwareAssignmentTags().isEmpty(); + if (hasRackAwareAssignmentTags) { + final BiConsumer>> addTags = (cs, tagSet) -> { + final Map tags = cs.clientTags(); + if (tags != null) { + tagSet.addAll(tags.entrySet().stream() + .map(entry -> KeyValue.pair(entry.getKey(), entry.getValue())) + .collect(Collectors.toList()) + ); + } + }; + + final Map clients = applicationState.kafkaStreamsStates(false); + + return (source, destination, sourceTask, kafkaStreamsAssignments) -> { + final Set> tagsWithSource = new HashSet<>(); + final Set> tagsWithDestination = new HashSet<>(); + for (final KafkaStreamsAssignment assignment: kafkaStreamsAssignments.values()) { + final boolean hasAssignedTask = assignment.tasks().containsKey(sourceTask); + final boolean isSourceProcess = assignment.processId().equals(source.processId()); + final boolean isDestinationProcess = assignment.processId().equals(destination.processId()); + if (hasAssignedTask && !isSourceProcess && !isDestinationProcess) { + final KafkaStreamsState clientState = clients.get(assignment.processId()); + addTags.accept(clientState, tagsWithSource); + addTags.accept(clientState, tagsWithDestination); + } + } + addTags.accept(source, tagsWithSource); + addTags.accept(destination, tagsWithDestination); + return tagsWithDestination.size() >= tagsWithSource.size(); + }; + } else { + return (a, b, c, d) -> true; + } + } + + private static ConstrainedPrioritySet standbyTaskPriorityListByLoad(final Map streamStates, + final Map kafkaStreamsAssignments) { + return new ConstrainedPrioritySet( + (processId, taskId) -> kafkaStreamsAssignments.get(new ProcessId(processId)).tasks().containsKey(taskId), + processId -> { + final double capacity = streamStates.get(new ProcessId(processId)).numProcessingThreads(); + final double numTasks = kafkaStreamsAssignments.get(new ProcessId(processId)).tasks().size(); + return numTasks / capacity; + } + ); + } + + private static void assignPendingStandbyTasksToLeastLoadedClients(final Map clients, + final int numStandbyReplicas, + final ConstrainedPrioritySet standbyTaskClientsByTaskLoad, + final Map pendingStandbyTaskToNumberRemainingStandbys) { + // We need to re offer all the clients to find the least loaded ones + standbyTaskClientsByTaskLoad.offerAll(clients.keySet()); + + for (final Map.Entry pendingStandbyTaskAssignmentEntry : pendingStandbyTaskToNumberRemainingStandbys.entrySet()) { + final TaskId activeTaskId = pendingStandbyTaskAssignmentEntry.getKey(); + + assignStandbyTasksForActiveTask( + numStandbyReplicas, + clients, + pendingStandbyTaskToNumberRemainingStandbys, + standbyTaskClientsByTaskLoad, + activeTaskId + ); + } + } + + private static class TagStatistics { + private final Map> tagKeyToValues; + private final Map, Set> tagEntryToClients; + + private TagStatistics(final Map> tagKeyToValues, + final Map, Set> tagEntryToClients) { + this.tagKeyToValues = tagKeyToValues; + this.tagEntryToClients = tagEntryToClients; + } + + public TagStatistics(final ApplicationState applicationState) { + final Map clientStates = applicationState.kafkaStreamsStates(false); + + final Map> tagKeyToValues = new HashMap<>(); + final Map, Set> tagEntryToClients = new HashMap<>(); + for (final KafkaStreamsState state : clientStates.values()) { + state.clientTags().forEach((tagKey, tagValue) -> { + tagKeyToValues.computeIfAbsent(tagKey, ignored -> new HashSet<>()).add(tagValue); + tagEntryToClients.computeIfAbsent(new KeyValue<>(tagKey, tagValue), ignored -> new HashSet<>()).add(state.processId()); + }); + } + + this.tagKeyToValues = tagKeyToValues; + this.tagEntryToClients = tagEntryToClients; + } } } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignor.java index acfbe4beb84e9..57ae9be304714 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskAssignor.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.processor.assignment; import java.util.Collection; +import java.util.Map; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.GroupAssignment; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.GroupSubscription; import org.apache.kafka.common.Configurable; @@ -36,6 +37,7 @@ public interface TaskAssignor extends Configurable { * ACTIVE_TASK_ASSIGNED_MULTIPLE_TIMES: multiple KafkaStreams clients assigned with the same active task * ACTIVE_AND_STANDBY_TASK_ASSIGNED_TO_SAME_KAFKASTREAMS: active task and standby task assigned to the same KafkaStreams client * INVALID_STANDBY_TASK: stateless task assigned as a standby task + * MISSING_PROCESS_ID: ProcessId present in the input ApplicationState was not present in the output TaskAssignment * UNKNOWN_PROCESS_ID: unrecognized ProcessId not matching any of the participating consumers * UNKNOWN_TASK_ID: unrecognized TaskId not matching any of the tasks to be assigned */ @@ -44,6 +46,7 @@ enum AssignmentError { ACTIVE_TASK_ASSIGNED_MULTIPLE_TIMES, ACTIVE_AND_STANDBY_TASK_ASSIGNED_TO_SAME_KAFKASTREAMS, INVALID_STANDBY_TASK, + MISSING_PROCESS_ID, UNKNOWN_PROCESS_ID, UNKNOWN_TASK_ID } @@ -73,6 +76,9 @@ enum AssignmentError { */ default void onAssignmentComputed(GroupAssignment assignment, GroupSubscription subscription, AssignmentError error) {} + @Override + default void configure(final Map configs) {} + /** * Wrapper class for the final assignment of active and standbys tasks to individual * KafkaStreams clients. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskInfo.java index 2e61c2a5b561a..22217b3bb2555 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskInfo.java @@ -17,9 +17,7 @@ package org.apache.kafka.streams.processor.assignment; -import java.util.Map; import java.util.Set; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.processor.TaskId; /** @@ -52,21 +50,7 @@ public interface TaskInfo { /** * - * @return the set of source topic partitions. This set will include both changelog and non-changelog - * topic partitions. + * @return the set of topic partitions in use for this task. */ - Set sourceTopicPartitions(); - - /** - * - * @return the set of changelog topic partitions. This set will include both source and non-source - * topic partitions. - */ - Set changelogTopicPartitions(); - - /** - * - * @return the mapping of {@code TopicPartition} to set of rack ids that this partition resides on. - */ - Map> partitionToRackIds(); -} \ No newline at end of file + Set topicPartitions(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java new file mode 100644 index 0000000000000..2174b6d823b6f --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java @@ -0,0 +1,465 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.processor.assignment.assignors; + +import static java.util.Collections.unmodifiableMap; + +import java.time.Instant; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.TreeSet; +import java.util.stream.Collectors; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.assignment.ApplicationState; +import org.apache.kafka.streams.processor.assignment.KafkaStreamsAssignment; +import org.apache.kafka.streams.processor.assignment.KafkaStreamsAssignment.AssignedTask; +import org.apache.kafka.streams.processor.assignment.KafkaStreamsState; +import org.apache.kafka.streams.processor.assignment.ProcessId; +import org.apache.kafka.streams.processor.assignment.TaskAssignmentUtils; +import org.apache.kafka.streams.processor.assignment.TaskAssignor; +import org.apache.kafka.streams.processor.assignment.TaskInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class StickyTaskAssignor implements TaskAssignor { + private static final Logger LOG = LoggerFactory.getLogger(StickyTaskAssignor.class); + + private final boolean mustPreserveActiveTaskAssignment; + + public StickyTaskAssignor() { + this(false); + } + + public StickyTaskAssignor(final boolean mustPreserveActiveTaskAssignment) { + this.mustPreserveActiveTaskAssignment = mustPreserveActiveTaskAssignment; + } + + @Override + public TaskAssignment assign(final ApplicationState applicationState) { + final Map clients = applicationState.kafkaStreamsStates(false); + final Map previousActiveAssignment = mapPreviousActiveTasks(clients); + final Map> previousStandbyAssignment = mapPreviousStandbyTasks(clients); + final AssignmentState assignmentState = new AssignmentState(applicationState, clients, + previousActiveAssignment, previousStandbyAssignment); + + assignActive(applicationState, clients.values(), assignmentState, this.mustPreserveActiveTaskAssignment); + optimizeActive(applicationState, assignmentState); + assignStandby(applicationState, assignmentState); + optimizeStandby(applicationState, assignmentState); + + final Map finalAssignments = assignmentState.buildKafkaStreamsAssignments(); + if (mustPreserveActiveTaskAssignment && !finalAssignments.isEmpty()) { + // We set the followup deadline for only one of the clients. + final ProcessId clientId = finalAssignments.entrySet().iterator().next().getKey(); + final KafkaStreamsAssignment previousAssignment = finalAssignments.get(clientId); + finalAssignments.put(clientId, previousAssignment.withFollowupRebalance(Instant.ofEpochMilli(0))); + } + + return new TaskAssignment(finalAssignments.values()); + } + + private void optimizeActive(final ApplicationState applicationState, + final AssignmentState assignmentState) { + if (mustPreserveActiveTaskAssignment) { + return; + } + + final Map currentAssignments = assignmentState.buildKafkaStreamsAssignments(); + + final Set statefulTasks = applicationState.allTasks().values().stream() + .filter(TaskInfo::isStateful) + .map(TaskInfo::id) + .collect(Collectors.toSet()); + final Map optimizedAssignmentsForStatefulTasks = TaskAssignmentUtils.optimizeRackAwareActiveTasks( + applicationState, currentAssignments, new TreeSet<>(statefulTasks)); + + final Set statelessTasks = applicationState.allTasks().values().stream() + .filter(task -> !task.isStateful()) + .map(TaskInfo::id) + .collect(Collectors.toSet()); + final Map optimizedAssignmentsForAllTasks = TaskAssignmentUtils.optimizeRackAwareActiveTasks( + applicationState, optimizedAssignmentsForStatefulTasks, new TreeSet<>(statelessTasks)); + + assignmentState.processOptimizedAssignments(optimizedAssignmentsForAllTasks); + } + + private void optimizeStandby(final ApplicationState applicationState, final AssignmentState assignmentState) { + if (applicationState.assignmentConfigs().numStandbyReplicas() <= 0) { + return; + } + + if (mustPreserveActiveTaskAssignment) { + return; + } + + final Map currentAssignments = assignmentState.buildKafkaStreamsAssignments(); + final Map optimizedAssignments = TaskAssignmentUtils.optimizeRackAwareStandbyTasks( + applicationState, currentAssignments); + assignmentState.processOptimizedAssignments(optimizedAssignments); + } + + private static void assignActive(final ApplicationState applicationState, + final Collection clients, + final AssignmentState assignmentState, + final boolean mustPreserveActiveTaskAssignment) { + final int totalCapacity = computeTotalProcessingThreads(clients); + final Set allTaskIds = applicationState.allTasks().keySet(); + final int taskCount = allTaskIds.size(); + final int activeTasksPerThread = taskCount / totalCapacity; + final Set unassigned = new HashSet<>(allTaskIds); + + // first try and re-assign existing active tasks to clients that previously had + // the same active task + for (final TaskId taskId : assignmentState.previousActiveAssignment.keySet()) { + final ProcessId previousClientForTask = assignmentState.previousActiveAssignment.get(taskId); + if (allTaskIds.contains(taskId)) { + if (mustPreserveActiveTaskAssignment || assignmentState.hasRoomForActiveTask(previousClientForTask, activeTasksPerThread)) { + assignmentState.finalizeAssignment(taskId, previousClientForTask, AssignedTask.Type.ACTIVE); + unassigned.remove(taskId); + } + } + } + + // try and assign any remaining unassigned tasks to clients that previously + // have seen the task. + for (final Iterator iterator = unassigned.iterator(); iterator.hasNext(); ) { + final TaskId taskId = iterator.next(); + final Set previousClientsForStandbyTask = assignmentState.previousStandbyAssignment.getOrDefault(taskId, new HashSet<>()); + for (final ProcessId client: previousClientsForStandbyTask) { + if (assignmentState.hasRoomForActiveTask(client, activeTasksPerThread)) { + assignmentState.finalizeAssignment(taskId, client, AssignedTask.Type.ACTIVE); + iterator.remove(); + break; + } + } + } + + // assign any remaining unassigned tasks + final List sortedTasks = new ArrayList<>(unassigned); + Collections.sort(sortedTasks); + for (final TaskId taskId : sortedTasks) { + final Set candidateClients = clients.stream() + .map(KafkaStreamsState::processId) + .collect(Collectors.toSet()); + final ProcessId bestClient = assignmentState.findBestClientForTask(taskId, candidateClients); + assignmentState.finalizeAssignment(taskId, bestClient, AssignedTask.Type.ACTIVE); + } + } + + private static void assignStandby(final ApplicationState applicationState, + final AssignmentState assignmentState) { + final Set statefulTasks = applicationState.allTasks().values().stream() + .filter(TaskInfo::isStateful) + .collect(Collectors.toSet()); + final int numStandbyReplicas = applicationState.assignmentConfigs().numStandbyReplicas(); + for (final TaskInfo task : statefulTasks) { + for (int i = 0; i < numStandbyReplicas; i++) { + final Set candidateClients = assignmentState.findClientsWithoutAssignedTask(task.id()); + if (candidateClients.isEmpty()) { + LOG.warn("Unable to assign {} of {} standby tasks for task [{}]. " + + "There is not enough available capacity. You should " + + "increase the number of threads and/or application instances " + + "to maintain the requested number of standby replicas.", + numStandbyReplicas - i, + numStandbyReplicas, task.id()); + break; + } + + final ProcessId bestClient = assignmentState.findBestClientForTask(task.id(), candidateClients); + assignmentState.finalizeAssignment(task.id(), bestClient, AssignedTask.Type.STANDBY); + } + } + } + + private static Map mapPreviousActiveTasks(final Map clients) { + final Map previousActiveTasks = new HashMap<>(); + for (final KafkaStreamsState client : clients.values()) { + for (final TaskId taskId : client.previousActiveTasks()) { + previousActiveTasks.put(taskId, client.processId()); + } + } + return previousActiveTasks; + } + + private static Map> mapPreviousStandbyTasks(final Map clients) { + final Map> previousStandbyTasks = new HashMap<>(); + for (final KafkaStreamsState client : clients.values()) { + for (final TaskId taskId : client.previousActiveTasks()) { + previousStandbyTasks.computeIfAbsent(taskId, k -> new HashSet<>()); + previousStandbyTasks.get(taskId).add(client.processId()); + } + } + return previousStandbyTasks; + } + + private static int computeTotalProcessingThreads(final Collection clients) { + int count = 0; + for (final KafkaStreamsState client : clients) { + count += client.numProcessingThreads(); + } + return count; + } + + private static class AssignmentState { + private final Map clients; + private final Map previousActiveAssignment; + private final Map> previousStandbyAssignment; + + private final TaskPairs taskPairs; + + private Map> newTaskLocations; + private Map> newAssignments; + + private AssignmentState(final ApplicationState applicationState, + final Map clients, + final Map previousActiveAssignment, + final Map> previousStandbyAssignment) { + this.clients = clients; + this.previousActiveAssignment = unmodifiableMap(previousActiveAssignment); + this.previousStandbyAssignment = unmodifiableMap(previousStandbyAssignment); + + final int taskCount = applicationState.allTasks().size(); + final int maxPairs = taskCount * (taskCount - 1) / 2; + this.taskPairs = new TaskPairs(maxPairs); + + this.newTaskLocations = new HashMap<>(); + this.newAssignments = new HashMap<>(); + } + + public void finalizeAssignment(final TaskId taskId, final ProcessId client, final AssignedTask.Type type) { + newAssignments.computeIfAbsent(client, k -> new HashSet<>()); + newTaskLocations.computeIfAbsent(taskId, k -> new HashSet<>()); + + final Set newAssignmentsForClient = newAssignments.get(client) + .stream().map(AssignedTask::id).collect(Collectors.toSet()); + + taskPairs.addPairs(taskId, newAssignmentsForClient); + newAssignments.get(client).add(new AssignedTask(taskId, type)); + newTaskLocations.get(taskId).add(client); + } + + public Map buildKafkaStreamsAssignments() { + final Map kafkaStreamsAssignments = new HashMap<>(); + for (final Map.Entry> entry : newAssignments.entrySet()) { + final ProcessId processId = entry.getKey(); + final Set assignedTasks = newAssignments.get(processId); + final KafkaStreamsAssignment assignment = KafkaStreamsAssignment.of(processId, assignedTasks); + kafkaStreamsAssignments.put(processId, assignment); + } + return kafkaStreamsAssignments; + } + + public void processOptimizedAssignments(final Map optimizedAssignments) { + final Map> newTaskLocations = new HashMap<>(); + final Map> newAssignments = new HashMap<>(); + + for (final Map.Entry entry : optimizedAssignments.entrySet()) { + final ProcessId processId = entry.getKey(); + final Set assignedTasks = new HashSet<>(optimizedAssignments.get(processId).tasks().values()); + newAssignments.put(processId, assignedTasks); + + for (final AssignedTask task : assignedTasks) { + newTaskLocations.computeIfAbsent(task.id(), k -> new HashSet<>()); + newTaskLocations.get(task.id()).add(processId); + } + } + + this.newTaskLocations = newTaskLocations; + this.newAssignments = newAssignments; + } + + public boolean hasRoomForActiveTask(final ProcessId processId, final int activeTasksPerThread) { + final int capacity = clients.get(processId).numProcessingThreads(); + final int newActiveTaskCount = newAssignments.computeIfAbsent(processId, k -> new HashSet<>()) + .stream().filter(assignedTask -> assignedTask.type() == AssignedTask.Type.ACTIVE) + .collect(Collectors.toSet()) + .size(); + return newActiveTaskCount < capacity * activeTasksPerThread; + } + + public ProcessId findBestClientForTask(final TaskId taskId, final Set clientsWithin) { + if (clientsWithin.size() == 1) { + return clientsWithin.iterator().next(); + } + + final ProcessId previousClient = findLeastLoadedClientWithPreviousActiveOrStandbyTask( + taskId, clientsWithin); + if (previousClient == null) { + return findLeastLoadedClient(taskId, clientsWithin); + } + + if (shouldBalanceLoad(previousClient)) { + final ProcessId standby = findLeastLoadedClientWithPreviousStandbyTask(taskId, clientsWithin); + if (standby == null || shouldBalanceLoad(standby)) { + return findLeastLoadedClient(taskId, clientsWithin); + } + return standby; + } + return previousClient; + } + + public Set findClientsWithoutAssignedTask(final TaskId taskId) { + final Set unavailableClients = newTaskLocations.computeIfAbsent(taskId, k -> new HashSet<>()); + return clients.values().stream() + .map(KafkaStreamsState::processId) + .filter(o -> !unavailableClients.contains(o)) + .collect(Collectors.toSet()); + } + + public double clientLoad(final ProcessId processId) { + final int capacity = clients.get(processId).numProcessingThreads(); + final double totalTaskCount = newAssignments.getOrDefault(processId, new HashSet<>()).size(); + return totalTaskCount / capacity; + } + + public ProcessId findLeastLoadedClient(final TaskId taskId, final Set clientIds) { + ProcessId leastLoaded = null; + for (final ProcessId processId : clientIds) { + final double thisClientLoad = clientLoad(processId); + if (thisClientLoad == 0) { + return processId; + } + + if (leastLoaded == null || thisClientLoad < clientLoad(leastLoaded)) { + final Set assignedTasks = newAssignments.getOrDefault(processId, new HashSet<>()) + .stream().map(AssignedTask::id).collect(Collectors.toSet()); + if (taskPairs.hasNewPair(taskId, assignedTasks)) { + leastLoaded = processId; + } + } + } + + if (leastLoaded != null) { + return leastLoaded; + } + + for (final ProcessId processId : clientIds) { + final double thisClientLoad = clientLoad(processId); + + if (leastLoaded == null || thisClientLoad < clientLoad(leastLoaded)) { + leastLoaded = processId; + } + } + + return leastLoaded; + } + + public ProcessId findLeastLoadedClientWithPreviousActiveOrStandbyTask(final TaskId taskId, + final Set clientsWithin) { + final ProcessId previous = previousActiveAssignment.get(taskId); + if (previous != null && clientsWithin.contains(previous)) { + return previous; + } + return findLeastLoadedClientWithPreviousStandbyTask(taskId, clientsWithin); + } + + public ProcessId findLeastLoadedClientWithPreviousStandbyTask(final TaskId taskId, + final Set clientsWithin) { + final Set ids = previousStandbyAssignment.getOrDefault(taskId, new HashSet<>()); + final HashSet constrainTo = new HashSet<>(ids); + constrainTo.retainAll(clientsWithin); + return findLeastLoadedClient(taskId, constrainTo); + } + + public boolean shouldBalanceLoad(final ProcessId client) { + final double thisClientLoad = clientLoad(client); + if (thisClientLoad < 1) { + return false; + } + + for (final ProcessId otherClient : clients.keySet()) { + if (clientLoad(otherClient) < thisClientLoad) { + return true; + } + } + return false; + } + } + + private static class TaskPairs { + private final Set pairs; + private final int maxPairs; + + public TaskPairs(final int maxPairs) { + this.maxPairs = maxPairs; + this.pairs = new HashSet<>(maxPairs); + } + + public boolean hasNewPair(final TaskId task1, + final Set taskIds) { + if (pairs.size() == maxPairs) { + return false; + } + for (final TaskId taskId : taskIds) { + if (!pairs.contains(pair(task1, taskId))) { + return true; + } + } + return false; + } + + public void addPairs(final TaskId taskId, final Set assigned) { + for (final TaskId id : assigned) { + pairs.add(pair(id, taskId)); + } + } + + public TaskPair pair(final TaskId task1, final TaskId task2) { + if (task1.compareTo(task2) < 0) { + return new TaskPair(task1, task2); + } + return new TaskPair(task2, task1); + } + } + + private static class TaskPair { + private final TaskId task1; + private final TaskId task2; + + TaskPair(final TaskId task1, final TaskId task2) { + this.task1 = task1; + this.task2 = task2; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final TaskPair pair = (TaskPair) o; + return Objects.equals(task1, pair.task1) && + Objects.equals(task2, pair.task2); + } + + @Override + public int hashCode() { + return Objects.hash(task1, task2); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index 82a10615293e4..658ba1540da48 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -16,7 +16,9 @@ */ package org.apache.kafka.streams.processor.internals; +import java.time.Instant; import java.util.Optional; +import java.util.function.Function; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.ListOffsetsResult; import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; @@ -38,10 +40,14 @@ import org.apache.kafka.streams.errors.TaskAssignmentException; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.assignment.ApplicationState; +import org.apache.kafka.streams.processor.assignment.KafkaStreamsAssignment; +import org.apache.kafka.streams.processor.assignment.KafkaStreamsState; +import org.apache.kafka.streams.processor.assignment.TaskAssignor.AssignmentError; import org.apache.kafka.streams.processor.assignment.TaskInfo; import org.apache.kafka.streams.processor.assignment.ProcessId; import org.apache.kafka.streams.processor.assignment.TaskAssignor.TaskAssignment; -import org.apache.kafka.streams.processor.internals.assignment.ApplicationStateImpl; +import org.apache.kafka.streams.processor.assignment.TaskTopicPartition; +import org.apache.kafka.streams.processor.internals.assignment.DefaultApplicationState; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; import org.apache.kafka.streams.processor.internals.assignment.AssignmentInfo; @@ -51,6 +57,7 @@ import org.apache.kafka.streams.processor.internals.assignment.AssignorError; import org.apache.kafka.streams.processor.internals.assignment.ClientState; import org.apache.kafka.streams.processor.internals.assignment.CopartitionedTopicsEnforcer; +import org.apache.kafka.streams.processor.internals.assignment.DefaultTaskTopicPartition; import org.apache.kafka.streams.processor.internals.assignment.FallbackPriorTaskAssignor; import org.apache.kafka.streams.processor.internals.assignment.RackAwareTaskAssignor; import org.apache.kafka.streams.processor.internals.assignment.RackUtils; @@ -185,6 +192,11 @@ public String toString() { } } + @FunctionalInterface + public interface UserTaskAssignmentListener { + void onAssignmentComputed(GroupAssignment assignment, GroupSubscription subscription); + } + // keep track of any future consumers in a "dummy" Client since we can't decipher their subscription private static final UUID FUTURE_ID = randomUUID(); @@ -212,6 +224,7 @@ public String toString() { private RebalanceProtocol rebalanceProtocol; private AssignmentListener assignmentListener; + private Supplier> userTaskAssignorSupplier; private Supplier taskAssignorSupplier; private byte uniqueField; private Map clientTags; @@ -246,6 +259,7 @@ public void configure(final Map configs) { internalTopicManager = assignorConfiguration.internalTopicManager(); copartitionedTopicsEnforcer = assignorConfiguration.copartitionedTopicsEnforcer(); rebalanceProtocol = assignorConfiguration.rebalanceProtocol(); + userTaskAssignorSupplier = assignorConfiguration::userTaskAssignor; taskAssignorSupplier = assignorConfiguration::taskAssignor; assignmentListener = assignorConfiguration.assignmentListener(); uniqueField = 0; @@ -398,9 +412,6 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr } try { - final boolean versionProbing = - checkMetadataVersions(minReceivedMetadataVersion, minSupportedMetadataVersion, futureMetadataVersion); - log.debug("Constructed client metadata {} from the member subscriptions.", clientMetadataMap); // ---------------- Step One ---------------- // @@ -438,7 +449,9 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr final Set statefulTasks = new HashSet<>(); - final boolean probingRebalanceNeeded = assignTasksToClients(fullMetadata, allSourceTopics, topicGroups, + final boolean versionProbing = + checkMetadataVersions(minReceivedMetadataVersion, minSupportedMetadataVersion, futureMetadataVersion); + final UserTaskAssignmentListener userTaskAssignmentListener = assignTasksToClients(fullMetadata, allSourceTopics, topicGroups, clientMetadataMap, partitionsForTask, racksForProcessConsumer, statefulTasks); // ---------------- Step Three ---------------- // @@ -463,11 +476,12 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr allOwnedPartitions, minReceivedMetadataVersion, minSupportedMetadataVersion, - versionProbing, - probingRebalanceNeeded + versionProbing ); - return new GroupAssignment(assignment); + final GroupAssignment groupAssignment = new GroupAssignment(assignment); + userTaskAssignmentListener.onAssignmentComputed(groupAssignment, groupSubscription); + return groupAssignment; } catch (final MissingSourceTopicException e) { log.error("Caught an error in the task assignment. Returning an error assignment.", e); return new GroupAssignment( @@ -513,54 +527,52 @@ private ApplicationState buildApplicationState(final TopologyMetadata topologyMe + "tasks for source topics vs changelog topics."); } - final Set sourceTopicPartitions = new HashSet<>(); - final Set nonSourceChangelogTopicPartitions = new HashSet<>(); - for (final Map.Entry> entry : sourcePartitionsForTask.entrySet()) { - final TaskId taskId = entry.getKey(); - final Set taskSourcePartitions = entry.getValue(); - final Set taskChangelogPartitions = changelogPartitionsForTask.get(taskId); - final Set taskNonSourceChangelogPartitions = new HashSet<>(taskChangelogPartitions); - taskNonSourceChangelogPartitions.removeAll(taskSourcePartitions); - - sourceTopicPartitions.addAll(taskSourcePartitions); - nonSourceChangelogTopicPartitions.addAll(taskNonSourceChangelogPartitions); - } + final Set logicalTaskIds = unmodifiableSet(sourcePartitionsForTask.keySet()); + final Set allTopicPartitions = new HashSet<>(); + final Map> topicPartitionsForTask = new HashMap<>(); + logicalTaskIds.forEach(taskId -> { + final Set topicPartitions = new HashSet<>(); + + for (final TopicPartition topicPartition : sourcePartitionsForTask.get(taskId)) { + final boolean isSource = true; + final boolean isChangelog = changelogPartitionsForTask.get(taskId).contains(topicPartition); + final DefaultTaskTopicPartition racklessTopicPartition = new DefaultTaskTopicPartition( + topicPartition, isSource, isChangelog, null); + allTopicPartitions.add(racklessTopicPartition); + topicPartitions.add(racklessTopicPartition); + } - final Map> racksForSourcePartitions = RackUtils.getRacksForTopicPartition( - cluster, internalTopicManager, sourceTopicPartitions, false); - final Map> racksForChangelogPartitions = RackUtils.getRacksForTopicPartition( - cluster, internalTopicManager, nonSourceChangelogTopicPartitions, true); + for (final TopicPartition topicPartition : changelogPartitionsForTask.get(taskId)) { + final boolean isSource = sourcePartitionsForTask.get(taskId).contains(topicPartition); + final boolean isChangelog = true; + final DefaultTaskTopicPartition racklessTopicPartition = new DefaultTaskTopicPartition( + topicPartition, isSource, isChangelog, null); + allTopicPartitions.add(racklessTopicPartition); + topicPartitions.add(racklessTopicPartition); + } - final Set logicalTaskIds = unmodifiableSet(sourcePartitionsForTask.keySet()); - final Set logicalTasks = logicalTaskIds.stream().map(taskId -> { - final Set stateStoreNames = topologyMetadata - .stateStoreNameToSourceTopicsForTopology(taskId.topologyName()) - .keySet(); - final Set sourcePartitions = sourcePartitionsForTask.get(taskId); - final Set changelogPartitions = changelogPartitionsForTask.get(taskId); - final Map> racksForTaskPartition = new HashMap<>(); - sourcePartitions.forEach(topicPartition -> { - racksForTaskPartition.put(topicPartition, racksForSourcePartitions.get(topicPartition)); - }); - changelogPartitions.forEach(topicPartition -> { - if (racksForSourcePartitions.containsKey(topicPartition)) { - racksForTaskPartition.put(topicPartition, racksForSourcePartitions.get(topicPartition)); - } else { - racksForTaskPartition.put(topicPartition, racksForChangelogPartitions.get(topicPartition)); - } - }); - - return new DefaultTaskInfo( - taskId, - !stateStoreNames.isEmpty(), - racksForTaskPartition, - stateStoreNames, - sourcePartitions, - changelogPartitions - ); - }).collect(Collectors.toSet()); + topicPartitionsForTask.put(taskId, topicPartitions); + }); + + RackUtils.annotateTopicPartitionsWithRackInfo(cluster, internalTopicManager, allTopicPartitions); + + final Map logicalTasks = logicalTaskIds.stream().collect(Collectors.toMap( + Function.identity(), + taskId -> { + final Set stateStoreNames = topologyMetadata + .stateStoreNameToSourceTopicsForTopology(taskId.topologyName()) + .keySet(); + final Set topicPartitions = topicPartitionsForTask.get(taskId); + return new DefaultTaskInfo( + taskId, + !stateStoreNames.isEmpty(), + stateStoreNames, + topicPartitions + ); + } + )); - return new ApplicationStateImpl( + return new DefaultApplicationState( assignmentConfigs.toPublicAssignmentConfigs(), logicalTasks, clientMetadataMap @@ -573,6 +585,9 @@ private static void processStreamsPartitionAssignment(final Map allSourceTopics, /** * Assigns a set of tasks to each client (Streams instance) using the configured task assignor, and also * populate the stateful tasks that have been assigned to the clients - * @return true if a probing rebalance should be triggered */ - private boolean assignTasksToClients(final Cluster fullMetadata, - final Set allSourceTopics, - final Map topicGroups, - final Map clientMetadataMap, - final Map> partitionsForTask, - final Map>> racksForProcessConsumer, - final Set statefulTasks) { + private UserTaskAssignmentListener assignTasksToClients(final Cluster fullMetadata, + final Set allSourceTopics, + final Map topicGroups, + final Map clientMetadataMap, + final Map> partitionsForTask, + final Map>> racksForProcessConsumer, + final Set statefulTasks) { if (!statefulTasks.isEmpty()) { throw new TaskAssignmentException("The stateful tasks should not be populated before assigning tasks to clients"); } @@ -763,23 +777,50 @@ private boolean assignTasksToClients(final Cluster fullMetadata, log.debug("Assigning tasks and {} standby replicas to client nodes {}", numStandbyReplicas(), clientStates); - final TaskAssignor taskAssignor = createTaskAssignor(lagComputationSuccessful); - - final RackAwareTaskAssignor rackAwareTaskAssignor = new RackAwareTaskAssignor( - fullMetadata, - partitionsForTask, - changelogTopics.changelogPartionsForTask(), - tasksForTopicGroup, - racksForProcessConsumer, - internalTopicManager, - assignmentConfigs, - time - ); - final boolean probingRebalanceNeeded = taskAssignor.assign(clientStates, - allTasks, - statefulTasks, - rackAwareTaskAssignor, - assignmentConfigs); + final Optional userTaskAssignor = + userTaskAssignorSupplier.get(); + final UserTaskAssignmentListener userTaskAssignmentListener; + if (userTaskAssignor.isPresent()) { + final ApplicationState applicationState = buildApplicationState( + taskManager.topologyMetadata(), + clientMetadataMap, + topicGroups, + fullMetadata + ); + final org.apache.kafka.streams.processor.assignment.TaskAssignor assignor = userTaskAssignor.get(); + final TaskAssignment taskAssignment = assignor.assign(applicationState); + final AssignmentError assignmentError = validateTaskAssignment(applicationState, taskAssignment); + processStreamsPartitionAssignment(clientMetadataMap, taskAssignment); + userTaskAssignmentListener = (assignment, subscription) -> assignor.onAssignmentComputed(assignment, subscription, assignmentError); + } else { + userTaskAssignmentListener = (assignment, subscription) -> { }; + final TaskAssignor taskAssignor = createTaskAssignor(lagComputationSuccessful); + final RackAwareTaskAssignor rackAwareTaskAssignor = new RackAwareTaskAssignor( + fullMetadata, + partitionsForTask, + changelogTopics.changelogPartionsForTask(), + tasksForTopicGroup, + racksForProcessConsumer, + internalTopicManager, + assignmentConfigs, + time + ); + final boolean probingRebalanceNeeded = taskAssignor.assign(clientStates, + allTasks, + statefulTasks, + rackAwareTaskAssignor, + assignmentConfigs); + if (probingRebalanceNeeded) { + // Arbitrarily choose the leader's client to be responsible for triggering the probing rebalance, + // note once we pick the first consumer within the process to trigger probing rebalance, other consumer + // would not set to trigger any more. + final ClientMetadata rebalanceClientMetadata = clientMetadataMap.get(taskManager.processId()); + if (rebalanceClientMetadata != null) { + final Instant rebalanceDeadline = Instant.ofEpochMilli(time.milliseconds() + probingRebalanceIntervalMs()); + rebalanceClientMetadata.state.setFollowupRebalanceDeadline(rebalanceDeadline); + } + } + } // Break this up into multiple logs to make sure the summary info gets through, which helps avoid // info loss for example due to long line truncation with large apps @@ -792,8 +833,7 @@ private boolean assignTasksToClients(final Cluster fullMetadata, .sorted(comparingByKey()) .map(entry -> entry.getKey() + "=" + entry.getValue().currentAssignment()) .collect(Collectors.joining(Utils.NL))); - - return probingRebalanceNeeded; + return userTaskAssignmentListener; } private TaskAssignor createTaskAssignor(final boolean lagComputationSuccessful) { @@ -951,9 +991,8 @@ private Map computeNewAssignment(final Set statefulT final Set allOwnedPartitions, final int minUserMetadataVersion, final int minSupportedMetadataVersion, - final boolean versionProbing, - final boolean shouldTriggerProbingRebalance) { - boolean rebalanceRequired = shouldTriggerProbingRebalance || versionProbing; + final boolean versionProbing) { + boolean rebalanceRequired = versionProbing; final Map assignment = new HashMap<>(); // within the client, distribute tasks to its owned consumers @@ -995,10 +1034,7 @@ private Map computeNewAssignment(final Set statefulT activeTaskAssignment.get(threadEntry.getKey()).addAll(threadEntry.getValue()); } - // Arbitrarily choose the leader's client to be responsible for triggering the probing rebalance, - // note once we pick the first consumer within the process to trigger probing rebalance, other consumer - // would not set to trigger any more. - final boolean encodeNextProbingRebalanceTime = shouldTriggerProbingRebalance && clientId.equals(taskManager.processId()); + final boolean isNextProbingRebalanceEncoded = clientMetadata.state.followupRebalanceDeadline().isPresent(); final boolean tasksRevoked = addClientAssignments( statefulTasks, @@ -1011,11 +1047,10 @@ private Map computeNewAssignment(final Set statefulT activeTaskAssignment, standbyTaskAssignment, minUserMetadataVersion, - minSupportedMetadataVersion, - encodeNextProbingRebalanceTime + minSupportedMetadataVersion ); - if (tasksRevoked || encodeNextProbingRebalanceTime) { + if (tasksRevoked || isNextProbingRebalanceEncoded) { rebalanceRequired = true; log.debug("Requested client {} to schedule a followup rebalance", clientId); } @@ -1059,12 +1094,12 @@ private boolean addClientAssignments(final Set statefulTasks, final Map> activeTaskAssignments, final Map> standbyTaskAssignments, final int minUserMetadataVersion, - final int minSupportedMetadataVersion, - final boolean probingRebalanceNeeded) { + final int minSupportedMetadataVersion) { boolean followupRebalanceRequiredForRevokedTasks = false; // We only want to encode a scheduled probing rebalance for a single member in this client - boolean shouldEncodeProbingRebalance = probingRebalanceNeeded; + final Optional followupRebalanceDeadline = clientMetadata.state.followupRebalanceDeadline(); + boolean shouldEncodeProbingRebalance = followupRebalanceDeadline.isPresent(); // Loop through the consumers and build their assignment for (final String consumer : clientMetadata.consumers) { @@ -1111,7 +1146,7 @@ private boolean addClientAssignments(final Set statefulTasks, // Don't bother to schedule a probing rebalance if an immediate one is already scheduled shouldEncodeProbingRebalance = false; } else if (shouldEncodeProbingRebalance) { - final long nextRebalanceTimeMs = time.milliseconds() + probingRebalanceIntervalMs(); + final long nextRebalanceTimeMs = followupRebalanceDeadline.get().toEpochMilli(); log.info("Requesting followup rebalance be scheduled by {} for {} to probe for caught-up replica tasks.", consumer, Utils.toLogDateTimeFormat(nextRebalanceTimeMs)); info.setNextRebalanceTime(nextRebalanceTimeMs); @@ -1525,6 +1560,75 @@ private void maybeScheduleFollowupRebalance(final long encodedNextScheduledRebal } } + private AssignmentError validateTaskAssignment(final ApplicationState applicationState, + final TaskAssignment taskAssignment) { + final Collection assignments = taskAssignment.assignment(); + final Map activeTasksInOutput = new HashMap<>(); + final Map standbyTasksInOutput = new HashMap<>(); + for (final KafkaStreamsAssignment assignment : assignments) { + final Set tasksForAssignment = new HashSet<>(); + for (final KafkaStreamsAssignment.AssignedTask task : assignment.tasks().values()) { + if (activeTasksInOutput.containsKey(task.id()) && task.type() == KafkaStreamsAssignment.AssignedTask.Type.ACTIVE) { + log.error("Assignment is invalid: active task {} was assigned to multiple KafkaStreams clients: {} and {}", + task.id(), assignment.processId().id(), activeTasksInOutput.get(task.id()).id()); + return AssignmentError.ACTIVE_TASK_ASSIGNED_MULTIPLE_TIMES; + } + + if (tasksForAssignment.contains(task.id())) { + log.error("Assignment is invalid: both an active and standby copy of task {} were assigned to KafkaStreams client {}", + task.id(), assignment.processId().id()); + return AssignmentError.ACTIVE_AND_STANDBY_TASK_ASSIGNED_TO_SAME_KAFKASTREAMS; + } + + tasksForAssignment.add(task.id()); + if (task.type() == KafkaStreamsAssignment.AssignedTask.Type.ACTIVE) { + activeTasksInOutput.put(task.id(), assignment.processId()); + } else { + standbyTasksInOutput.put(task.id(), assignment.processId()); + } + } + } + + for (final TaskInfo task : applicationState.allTasks().values()) { + if (!task.isStateful() && standbyTasksInOutput.containsKey(task.id())) { + log.error("Assignment is invalid: standby task for stateless task {} was assigned to KafkaStreams client {}", + task.id(), standbyTasksInOutput.get(task.id()).id()); + return AssignmentError.INVALID_STANDBY_TASK; + } + } + + final Map clientStates = applicationState.kafkaStreamsStates(false); + final Set clientsInOutput = assignments.stream().map(KafkaStreamsAssignment::processId) + .collect(Collectors.toSet()); + for (final Map.Entry entry : clientStates.entrySet()) { + final ProcessId processIdInInput = entry.getKey(); + if (!clientsInOutput.contains(processIdInInput)) { + log.error("Assignment is invalid: KafkaStreams client {} has no assignment", processIdInInput.id()); + return AssignmentError.MISSING_PROCESS_ID; + } + } + + for (final ProcessId processIdInOutput : clientsInOutput) { + if (!clientStates.containsKey(processIdInOutput)) { + log.error("Assignment is invalid: the KafkaStreams client {} is unknown", processIdInOutput.id()); + return AssignmentError.UNKNOWN_PROCESS_ID; + } + } + + final Set taskIdsInInput = applicationState.allTasks().keySet(); + for (final KafkaStreamsAssignment assignment : assignments) { + for (final KafkaStreamsAssignment.AssignedTask task : assignment.tasks().values()) { + if (!taskIdsInInput.contains(task.id())) { + log.error("Assignment is invalid: task {} assigned to KafkaStreams client {} was unknown", + task.id(), assignment.processId().id()); + return AssignmentError.UNKNOWN_TASK_ID; + } + } + } + + return AssignmentError.NONE; + } + /** * Verify that this client's host info was included in the map returned in the assignment, and trigger a * rebalance if not. This may be necessary when using static membership, as a rejoining client will be handed diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java index 052ad117af467..176c475ffd9ee 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; +import java.util.Optional; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol; import org.apache.kafka.common.KafkaException; @@ -253,6 +254,24 @@ public TaskAssignor taskAssignor() { } } + public Optional userTaskAssignor() { + final String userTaskAssignorClassname = streamsConfig.getString(StreamsConfig.TASK_ASSIGNOR_CLASS_CONFIG); + if (userTaskAssignorClassname == null) { + return Optional.empty(); + } + try { + final org.apache.kafka.streams.processor.assignment.TaskAssignor assignor = Utils.newInstance(userTaskAssignorClassname, + org.apache.kafka.streams.processor.assignment.TaskAssignor.class); + log.info("Instantiated {} as the task assignor.", userTaskAssignorClassname); + return Optional.of(assignor); + } catch (final ClassNotFoundException e) { + throw new IllegalArgumentException( + "Expected an instantiable class name for " + StreamsConfig.TASK_ASSIGNOR_CLASS_CONFIG + " but got " + userTaskAssignorClassname, + e + ); + } + } + public AssignmentListener assignmentListener() { final Object o = internalConfigs.get(InternalConfig.ASSIGNMENT_LISTENER); if (o == null) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java index 1d9d8c47a4e08..704c1d50885c4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams.processor.internals.assignment; +import java.time.Instant; +import java.util.Optional; import java.util.SortedMap; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.processor.TaskId; @@ -62,6 +64,8 @@ public class ClientState { private final ClientStateTask previousStandbyTasks = new ClientStateTask(null, null); private final ClientStateTask revokingActiveTasks = new ClientStateTask(null, new TreeMap<>()); private final UUID processId; + + private Optional followupRebalanceDeadline = Optional.empty(); private int capacity; public ClientState() { @@ -81,8 +85,8 @@ public ClientState(final UUID processId, final Map clientTags) { } ClientState(final UUID processId, final int capacity, final Map clientTags) { - previousStandbyTasks.taskIds(new TreeSet<>()); - previousActiveTasks.taskIds(new TreeSet<>()); + previousStandbyTasks.setTaskIds(new TreeSet<>()); + previousActiveTasks.setTaskIds(new TreeSet<>()); taskOffsetSums = new TreeMap<>(); taskLagTotals = new TreeMap<>(); this.capacity = capacity; @@ -106,8 +110,8 @@ public ClientState(final Set previousActiveTasks, final Map clientTags, final int capacity, final UUID processId) { - this.previousStandbyTasks.taskIds(unmodifiableSet(new TreeSet<>(previousStandbyTasks))); - this.previousActiveTasks.taskIds(unmodifiableSet(new TreeSet<>(previousActiveTasks))); + this.previousStandbyTasks.setTaskIds(unmodifiableSet(new TreeSet<>(previousStandbyTasks))); + this.previousActiveTasks.setTaskIds(unmodifiableSet(new TreeSet<>(previousActiveTasks))); taskOffsetSums = emptyMap(); this.taskLagTotals = unmodifiableMap(taskLagTotals); this.capacity = capacity; @@ -143,6 +147,14 @@ boolean reachedCapacity() { return assignedTaskCount() >= capacity; } + public Optional followupRebalanceDeadline() { + return followupRebalanceDeadline; + } + + public void setFollowupRebalanceDeadline(final Instant followupRebalanceDeadline) { + this.followupRebalanceDeadline = Optional.of(followupRebalanceDeadline); + } + public Set activeTasks() { return unmodifiableSet(assignedActiveTasks.taskIds()); } @@ -477,14 +489,14 @@ public SortedMap> taskIdsByPreviousConsumer() { } public void setAssignedTasks(final KafkaStreamsAssignment assignment) { - final Set activeTasks = assignment.assignment().stream() + final Set activeTasks = assignment.tasks().values().stream() .filter(task -> task.type() == ACTIVE).map(KafkaStreamsAssignment.AssignedTask::id) .collect(Collectors.toSet()); - final Set standbyTasks = assignment.assignment().stream() + final Set standbyTasks = assignment.tasks().values().stream() .filter(task -> task.type() == STANDBY).map(KafkaStreamsAssignment.AssignedTask::id) .collect(Collectors.toSet()); - assignedActiveTasks.taskIds(activeTasks); - assignedStandbyTasks.taskIds(standbyTasks); + assignedActiveTasks.setTaskIds(activeTasks); + assignedStandbyTasks.setTaskIds(standbyTasks); } public String currentAssignment() { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTask.java index 92769699ccccd..1f098dc9aea81 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTask.java @@ -31,7 +31,7 @@ class ClientStateTask { this.consumerToTaskIds = consumerToTaskIds; } - void taskIds(final Set clientToTaskIds) { + void setTaskIds(final Set clientToTaskIds) { taskIds = clientToTaskIds; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ConstrainedPrioritySet.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ConstrainedPrioritySet.java index 1de9dfc3c67b9..3705ee39af37d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ConstrainedPrioritySet.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ConstrainedPrioritySet.java @@ -30,14 +30,14 @@ /** * Wraps a priority queue of clients and returns the next valid candidate(s) based on the current task assignment */ -class ConstrainedPrioritySet { +public class ConstrainedPrioritySet { private final PriorityQueue clientsByTaskLoad; private final BiFunction constraint; private final Set uniqueClients = new HashSet<>(); - ConstrainedPrioritySet(final BiFunction constraint, - final Function weight) { + public ConstrainedPrioritySet(final BiFunction constraint, + final Function weight) { this.constraint = constraint; clientsByTaskLoad = new PriorityQueue<>(Comparator.comparing(weight).thenComparing(clientId -> clientId)); } @@ -45,7 +45,7 @@ class ConstrainedPrioritySet { /** * @return the next least loaded client that satisfies the given criteria, or null if none do */ - UUID poll(final TaskId task, final Function extraConstraint) { + public UUID poll(final TaskId task, final Function extraConstraint) { final Set invalidPolledClients = new HashSet<>(); while (!clientsByTaskLoad.isEmpty()) { final UUID candidateClient = pollNextClient(); @@ -66,17 +66,17 @@ UUID poll(final TaskId task, final Function extraConstraint) { /** * @return the next least loaded client that satisfies the given criteria, or null if none do */ - UUID poll(final TaskId task) { + public UUID poll(final TaskId task) { return poll(task, client -> true); } - void offerAll(final Collection clients) { + public void offerAll(final Collection clients) { for (final UUID client : clients) { offer(client); } } - void offer(final UUID client) { + public void offer(final UUID client) { if (uniqueClients.contains(client)) { clientsByTaskLoad.remove(client); } else { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ApplicationStateImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultApplicationState.java similarity index 78% rename from streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ApplicationStateImpl.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultApplicationState.java index 41479f756e4e1..b3b3084bc1751 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ApplicationStateImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultApplicationState.java @@ -16,12 +16,11 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import static java.util.Collections.unmodifiableSet; +import static java.util.Collections.unmodifiableMap; import java.util.HashMap; import java.util.Map; import java.util.Optional; -import java.util.Set; import java.util.UUID; import org.apache.kafka.streams.processor.assignment.TaskInfo; import org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.ClientMetadata; @@ -32,22 +31,29 @@ import org.apache.kafka.streams.processor.assignment.ProcessId; import org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor; -public class ApplicationStateImpl implements ApplicationState { +public class DefaultApplicationState implements ApplicationState { private final AssignmentConfigs assignmentConfigs; - private final Set tasks; + private final Map tasks; private final Map clientStates; - public ApplicationStateImpl(final AssignmentConfigs assignmentConfigs, - final Set tasks, - final Map clientStates) { + private final Map> cachedKafkaStreamStates; + + public DefaultApplicationState(final AssignmentConfigs assignmentConfigs, + final Map tasks, + final Map clientStates) { this.assignmentConfigs = assignmentConfigs; - this.tasks = unmodifiableSet(tasks); + this.tasks = unmodifiableMap(tasks); this.clientStates = clientStates; + this.cachedKafkaStreamStates = new HashMap<>(); } @Override public Map kafkaStreamsStates(final boolean computeTaskLags) { + if (cachedKafkaStreamStates.containsKey(computeTaskLags)) { + return cachedKafkaStreamStates.get(computeTaskLags); + } + final Map kafkaStreamsStates = new HashMap<>(); for (final Map.Entry clientEntry : clientStates.entrySet()) { final ClientMetadata metadata = clientEntry.getValue(); @@ -68,6 +74,7 @@ public Map kafkaStreamsStates(final boolean comput kafkaStreamsStates.put(processId, kafkaStreamsState); } + cachedKafkaStreamStates.put(computeTaskLags, kafkaStreamsStates); return kafkaStreamsStates; } @@ -77,7 +84,7 @@ public AssignmentConfigs assignmentConfigs() { } @Override - public Set allTasks() { + public Map allTasks() { return tasks; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskInfo.java index c0212db862af2..14e53440f4bf8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskInfo.java @@ -16,36 +16,28 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import static java.util.Collections.unmodifiableMap; import static java.util.Collections.unmodifiableSet; -import java.util.Map; import java.util.Set; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.assignment.TaskInfo; +import org.apache.kafka.streams.processor.assignment.TaskTopicPartition; public class DefaultTaskInfo implements TaskInfo { private final TaskId id; private final boolean isStateful; - private final Map> partitionToRackIds; private final Set stateStoreNames; - private final Set sourceTopicPartitions; - private final Set changelogTopicPartitions; + private final Set topicPartitions; public DefaultTaskInfo(final TaskId id, final boolean isStateful, - final Map> partitionToRackIds, final Set stateStoreNames, - final Set sourceTopicPartitions, - final Set changelogTopicPartitions) { + final Set topicPartitions) { this.id = id; - this.partitionToRackIds = unmodifiableMap(partitionToRackIds); this.isStateful = isStateful; this.stateStoreNames = unmodifiableSet(stateStoreNames); - this.sourceTopicPartitions = unmodifiableSet(sourceTopicPartitions); - this.changelogTopicPartitions = unmodifiableSet(changelogTopicPartitions); + this.topicPartitions = unmodifiableSet(topicPartitions); } @Override @@ -64,17 +56,7 @@ public Set stateStoreNames() { } @Override - public Set sourceTopicPartitions() { - return sourceTopicPartitions; - } - - @Override - public Set changelogTopicPartitions() { - return changelogTopicPartitions; - } - - @Override - public Map> partitionToRackIds() { - return partitionToRackIds; + public Set topicPartitions() { + return topicPartitions; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskTopicPartition.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskTopicPartition.java index 815aa1ff64c92..1c0a640d9c4c9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskTopicPartition.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskTopicPartition.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; +import java.util.Objects; import java.util.Optional; import java.util.Set; import org.apache.kafka.common.TopicPartition; @@ -35,7 +36,9 @@ public class DefaultTaskTopicPartition implements TaskTopicPartition { private final TopicPartition topicPartition; private final boolean isSourceTopic; private final boolean isChangelogTopic; - private final Optional> rackIds; + + private Optional> rackIds; + public DefaultTaskTopicPartition(final TopicPartition topicPartition, final boolean isSourceTopic, @@ -66,4 +69,31 @@ public boolean isChangelog() { public Optional> rackIds() { return rackIds; } + + @Override + public int hashCode() { + int result = topicPartition.hashCode(); + result = 31 * result + Objects.hashCode(isSourceTopic); + result = 31 * result + Objects.hashCode(isChangelogTopic); + return result; + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + final TaskTopicPartition other = (TaskTopicPartition) obj; + return topicPartition.equals(other.topicPartition()) && + isSourceTopic == other.isSource() && + isChangelogTopic == other.isChangelog() && + rackIds.equals(other.rackIds()); + } + + public void annotateWithRackIds(final Set rackIds) { + this.rackIds = Optional.of(rackIds); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackAwareGraphConstructor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackAwareGraphConstructor.java index bf4582a6f0f35..538650216e356 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackAwareGraphConstructor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackAwareGraphConstructor.java @@ -32,6 +32,7 @@ /** * Construct graph for rack aware task assignor + * @param represents a KafkaStreams client and its currently-assigned tasks */ public interface RackAwareGraphConstructor { int SOURCE_ID = -1; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackAwareGraphConstructorFactory.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackAwareGraphConstructorFactory.java index 9dacfa34fd6db..0ae391fc29118 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackAwareGraphConstructorFactory.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackAwareGraphConstructorFactory.java @@ -17,6 +17,8 @@ package org.apache.kafka.streams.processor.internals.assignment; import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -30,12 +32,27 @@ public class RackAwareGraphConstructorFactory { static RackAwareGraphConstructor create(final AssignmentConfigs assignmentConfigs, final Map> tasksForTopicGroup) { - return create(assignmentConfigs, new ArrayList<>(new TreeMap<>(tasksForTopicGroup).values())); + return create(assignmentConfigs.rackAwareAssignmentStrategy, new ArrayList<>(new TreeMap<>(tasksForTopicGroup).values())); } - static RackAwareGraphConstructor create(final AssignmentConfigs assignmentConfigs, - final List> taskSetsPerTopicGroup) { - switch (assignmentConfigs.rackAwareAssignmentStrategy) { + public static RackAwareGraphConstructor create(final String rackAwareAssignmentStrategy, + final Collection allTasks) { + final Map> tasksForTopologyId = new TreeMap<>(); + allTasks.forEach(taskId -> { + tasksForTopologyId.computeIfAbsent(taskId.subtopology(), k -> new HashSet<>()); + tasksForTopologyId.get(taskId.subtopology()).add(taskId); + }); + + final List> tasksForTopicGroup = new ArrayList<>(); + tasksForTopologyId.forEach((subtopology, taskIds) -> { + tasksForTopicGroup.add(taskIds); + }); + return create(rackAwareAssignmentStrategy, tasksForTopicGroup); + } + + public static RackAwareGraphConstructor create(final String rackAwareAssignmentStrategy, + final List> taskSetsPerTopicGroup) { + switch (rackAwareAssignmentStrategy) { case StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC: return new MinTrafficGraphConstructor(); case StreamsConfig.RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY: @@ -44,5 +61,4 @@ static RackAwareGraphConstructor create(final AssignmentConfigs assignmen throw new IllegalArgumentException("Rack aware assignment is disabled"); } } - } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackAwareTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackAwareTaskAssignor.java index b10b51f7f15b9..4b430fbb2c1dd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackAwareTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackAwareTaskAssignor.java @@ -77,7 +77,7 @@ int getCost(final TaskId taskId, // This is number is picked based on testing. Usually the optimization for standby assignment // stops after 3 rounds - private static final int STANDBY_OPTIMIZER_MAX_ITERATION = 4; + public static final int STANDBY_OPTIMIZER_MAX_ITERATION = 4; private final Cluster fullMetadata; private final Map> partitionsForTask; @@ -134,17 +134,7 @@ public synchronized boolean canEnableRackAwareAssignor() { } // Visible for testing. This method also checks if all TopicPartitions exist in cluster - public boolean populateTopicsToDescribe(final Set topicsToDescribe, - final boolean changelog) { - return populateTopicsToDescribe(fullMetadata, changelogPartitionsForTask, partitionsForTask, changelog, topicsToDescribe, racksForPartition); - } - - public static boolean populateTopicsToDescribe(final Cluster fullMetadata, - final Map> changelogPartitionsForTask, - final Map> partitionsForTask, - final boolean changelog, - final Set topicsToDescribe, - final Map> racksForPartition) { + boolean populateTopicsToDescribe(final Set topicsToDescribe, final boolean changelog) { if (changelog) { // Changelog topics are not in metadata, we need to describe them changelogPartitionsForTask.values().stream().flatMap(Collection::stream).forEach(tp -> topicsToDescribe.add(tp.topic())); @@ -177,10 +167,6 @@ public static boolean populateTopicsToDescribe(final Cluster fullMetadata, return true; } - private boolean validateTopicPartitionRack(final boolean changelogTopics) { - return validateTopicPartitionRack(fullMetadata, internalTopicManager, changelogPartitionsForTask, partitionsForTask, changelogTopics, racksForPartition); - } - /** * This function populates the {@param racksForPartition} parameter passed into the function by using both * the {@code Cluster} metadata as well as the {@param internalTopicManager} for topics that have stale @@ -188,15 +174,10 @@ private boolean validateTopicPartitionRack(final boolean changelogTopics) { * * @return whether the operation successfully completed and the rack information is valid. */ - public static boolean validateTopicPartitionRack(final Cluster fullMetadata, - final InternalTopicManager internalTopicManager, - final Map> changelogPartitionsForTask, - final Map> partitionsForTask, - final boolean changelogTopics, - final Map> racksForPartition) { + private boolean validateTopicPartitionRack(final boolean changelogTopics) { // Make sure rackId exist for all TopicPartitions needed final Set topicsToDescribe = new HashSet<>(); - if (!populateTopicsToDescribe(fullMetadata, changelogPartitionsForTask, partitionsForTask, changelogTopics, topicsToDescribe, racksForPartition)) { + if (!populateTopicsToDescribe(topicsToDescribe, changelogTopics)) { return false; } @@ -486,12 +467,12 @@ public long optimizeStandbyTasks(final SortedMap clientStates .sorted() .collect(Collectors.toList()); - final Map taskClientMap = new HashMap<>(); final List clients = Stream.of(clientList.get(i), clientList.get(j)) .sorted().collect( Collectors.toList()); - final Map originalAssignedTaskNumber = new HashMap<>(); + final Map taskClientMap = new HashMap<>(); + final Map originalAssignedTaskNumber = new HashMap<>(); final Graph graph = graphConstructor.constructTaskGraph( clients, taskIdList, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackUtils.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackUtils.java index b3554c36b03b1..f9961c555fa6c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackUtils.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.streams.processor.assignment.TaskTopicPartition; import org.apache.kafka.streams.processor.internals.InternalTopicManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,26 +39,37 @@ public final class RackUtils { private RackUtils() { } - public static Map> getRacksForTopicPartition(final Cluster cluster, - final InternalTopicManager internalTopicManager, - final Set topicPartitions, - final boolean isChangelog) { - final Set topicsToDescribe = new HashSet<>(); - if (isChangelog) { - topicsToDescribe.addAll(topicPartitions.stream().map(TopicPartition::topic).collect( - Collectors.toSet())); - } else { - topicsToDescribe.addAll(topicsWithMissingMetadata(cluster, topicPartitions)); - } + public static void annotateTopicPartitionsWithRackInfo(final Cluster cluster, + final InternalTopicManager internalTopicManager, + final Set topicPartitions) { + // First we add all the changelog topics to the set of topics to describe. + final Set topicsToDescribe = topicPartitions.stream() + .filter(DefaultTaskTopicPartition::isChangelog) + .map(topicPartition -> topicPartition.topicPartition().topic()) + .collect(Collectors.toSet()); - final Set topicsWithUpToDateMetadata = topicPartitions.stream() - .filter(partition -> !topicsToDescribe.contains(partition.topic())) + // Then we add the non changelog topics that we do not have full information about. + final Set nonChangelogTopics = topicPartitions.stream() + .filter(taskTopicPartition -> !taskTopicPartition.isChangelog()) + .map(TaskTopicPartition::topicPartition) .collect(Collectors.toSet()); - final Map> racksForTopicPartition = knownRacksForPartition( - cluster, topicsWithUpToDateMetadata); + topicsToDescribe.addAll(topicsWithMissingMetadata(cluster, nonChangelogTopics)); + // We can issue an RPC call to get up-to-date information about the topics that had rack + // information missing. final Map> freshTopicPartitionInfo = describeTopics(internalTopicManager, topicsToDescribe); + + // Finally we compute the list of topics that already have all rack information known. + final Set topicsWithUpToDateMetadata = topicPartitions.stream() + .map(TaskTopicPartition::topicPartition) + .filter(topicPartition -> !topicsToDescribe.contains(topicPartition.topic())) + .collect(Collectors.toSet()); + + // Lastly we compile the mapping of topic partition to rack ids by combining known data and + // information that we got from the earlier RPC call. + final Map> racksForTopicPartition = knownRacksForPartition( + cluster, topicsWithUpToDateMetadata); freshTopicPartitionInfo.forEach((topic, partitionInfos) -> { for (final TopicPartitionInfo partitionInfo : partitionInfos) { final int partition = partitionInfo.partition(); @@ -75,7 +87,14 @@ public static Map> getRacksForTopicPartition(final C } }); - return racksForTopicPartition; + for (final DefaultTaskTopicPartition topicPartition : topicPartitions) { + if (!racksForTopicPartition.containsKey(topicPartition.topicPartition())) { + continue; + } + + final Set racks = racksForTopicPartition.get(topicPartition.topicPartition()); + topicPartition.annotateWithRackIds(racks); + } } public static Set topicsWithMissingMetadata(final Cluster cluster, final Set topicPartitions) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValue.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValue.java index bb7b516eddc39..5f5e0ab7dd717 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValue.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValue.java @@ -63,21 +63,6 @@ public static LeftOrRightValue makeRightValue(final V2 rightVal return new LeftOrRightValue<>(null, rightValue); } - /** - * Create a new {@link LeftOrRightValue} instance with the V value as {@code leftValue} if - * {@code isLeftSide} is True; otherwise {@code rightValue} if {@code isLeftSide} is False. - * - * @param value the V value (either V1 or V2 type) - * @param the type of the value - * @return a new {@link LeftOrRightValue} instance - */ - public static LeftOrRightValue make(final boolean isLeftSide, final V value) { - Objects.requireNonNull(value, "value is null"); - return isLeftSide - ? LeftOrRightValue.makeLeftValue(value) - : LeftOrRightValue.makeRightValue(value); - } - public V1 getLeftValue() { return leftValue; } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredIterator.java new file mode 100644 index 0000000000000..c04fb43020c53 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredIterator.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +/** + * Common super-interface of all Metered Iterator types. + * + * This enables tracking the timestamp the Iterator was first created, for the oldest-iterator-open-since-ms metric. + */ +public interface MeteredIterator { + + /** + * @return The UNIX timestamp, in milliseconds, that this Iterator was created/opened. + */ + long startTimestamp(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java index 88b49cd21803e..fbe42b87065e8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java @@ -49,10 +49,13 @@ import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics; import java.util.ArrayList; +import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.NavigableSet; import java.util.Objects; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.LongAdder; import java.util.function.Function; import static org.apache.kafka.common.utils.Utils.mkEntry; @@ -90,11 +93,13 @@ public class MeteredKeyValueStore private Sensor prefixScanSensor; private Sensor flushSensor; private Sensor e2eLatencySensor; + protected Sensor iteratorDurationSensor; protected InternalProcessorContext context; private StreamsMetricsImpl streamsMetrics; private TaskId taskId; - protected AtomicInteger numOpenIterators = new AtomicInteger(0); + protected LongAdder numOpenIterators = new LongAdder(); + protected NavigableSet openIterators = new ConcurrentSkipListSet<>(Comparator.comparingLong(MeteredIterator::startTimestamp)); @SuppressWarnings("rawtypes") private final Map queryHandlers = @@ -165,8 +170,12 @@ private void registerMetrics() { flushSensor = StateStoreMetrics.flushSensor(taskId.toString(), metricsScope, name(), streamsMetrics); deleteSensor = StateStoreMetrics.deleteSensor(taskId.toString(), metricsScope, name(), streamsMetrics); e2eLatencySensor = StateStoreMetrics.e2ELatencySensor(taskId.toString(), metricsScope, name(), streamsMetrics); + iteratorDurationSensor = StateStoreMetrics.iteratorDurationSensor(taskId.toString(), metricsScope, name(), streamsMetrics); StateStoreMetrics.addNumOpenIteratorsGauge(taskId.toString(), metricsScope, name(), streamsMetrics, - (config, now) -> numOpenIterators.get()); + (config, now) -> numOpenIterators.sum()); + StateStoreMetrics.addOldestOpenIteratorGauge(taskId.toString(), metricsScope, name(), streamsMetrics, + (config, now) -> openIterators.isEmpty() ? null : openIterators.first().startTimestamp() + ); } protected Serde prepareValueSerdeForStore(final Serde valueSerde, final SerdeGetter getter) { @@ -454,18 +463,26 @@ protected void maybeRecordE2ELatency() { } } - private class MeteredKeyValueIterator implements KeyValueIterator { + private class MeteredKeyValueIterator implements KeyValueIterator, MeteredIterator { private final KeyValueIterator iter; private final Sensor sensor; private final long startNs; + private final long startTimestamp; private MeteredKeyValueIterator(final KeyValueIterator iter, final Sensor sensor) { this.iter = iter; this.sensor = sensor; + this.startTimestamp = time.milliseconds(); this.startNs = time.nanoseconds(); - numOpenIterators.incrementAndGet(); + numOpenIterators.increment(); + openIterators.add(this); + } + + @Override + public long startTimestamp() { + return startTimestamp; } @Override @@ -486,8 +503,11 @@ public void close() { try { iter.close(); } finally { - sensor.record(time.nanoseconds() - startNs); - numOpenIterators.decrementAndGet(); + final long duration = time.nanoseconds() - startNs; + sensor.record(duration); + iteratorDurationSensor.record(duration); + numOpenIterators.decrement(); + openIterators.remove(this); } } @@ -497,11 +517,12 @@ public K peekNextKey() { } } - private class MeteredKeyValueTimestampedIterator implements KeyValueIterator { + private class MeteredKeyValueTimestampedIterator implements KeyValueIterator, MeteredIterator { private final KeyValueIterator iter; private final Sensor sensor; private final long startNs; + private final long startTimestamp; private final Function valueDeserializer; private MeteredKeyValueTimestampedIterator(final KeyValueIterator iter, @@ -510,8 +531,15 @@ private MeteredKeyValueTimestampedIterator(final KeyValueIterator this.iter = iter; this.sensor = sensor; this.valueDeserializer = valueDeserializer; + this.startTimestamp = time.milliseconds(); this.startNs = time.nanoseconds(); - numOpenIterators.incrementAndGet(); + numOpenIterators.increment(); + openIterators.add(this); + } + + @Override + public long startTimestamp() { + return startTimestamp; } @Override @@ -532,8 +560,11 @@ public void close() { try { iter.close(); } finally { - sensor.record(time.nanoseconds() - startNs); - numOpenIterators.decrementAndGet(); + final long duration = time.nanoseconds() - startNs; + sensor.record(duration); + iteratorDurationSensor.record(duration); + numOpenIterators.decrement(); + openIterators.remove(this); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredMultiVersionedKeyQueryIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredMultiVersionedKeyQueryIterator.java index be695501cafde..48347365756c9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredMultiVersionedKeyQueryIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredMultiVersionedKeyQueryIterator.java @@ -16,33 +16,57 @@ */ package org.apache.kafka.streams.state.internals; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.LongAdder; +import java.util.Set; import java.util.function.Function; + +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.state.VersionedRecordIterator; import org.apache.kafka.streams.state.VersionedRecord; -public class MeteredMultiVersionedKeyQueryIterator implements VersionedRecordIterator { +class MeteredMultiVersionedKeyQueryIterator implements VersionedRecordIterator, MeteredIterator { private final VersionedRecordIterator iterator; private final Function, VersionedRecord> deserializeValue; - private final AtomicInteger numOpenIterators; + private final LongAdder numOpenIterators; + private final Sensor sensor; + private final Time time; + private final long startNs; + private final long startTimestampMs; + private final Set openIterators; public MeteredMultiVersionedKeyQueryIterator(final VersionedRecordIterator iterator, + final Sensor sensor, + final Time time, final Function, VersionedRecord> deserializeValue, - final AtomicInteger numOpenIterators) { + final LongAdder numOpenIterators, + final Set openIterators) { this.iterator = iterator; this.deserializeValue = deserializeValue; this.numOpenIterators = numOpenIterators; - numOpenIterators.incrementAndGet(); + this.openIterators = openIterators; + this.sensor = sensor; + this.time = time; + this.startNs = time.nanoseconds(); + this.startTimestampMs = time.milliseconds(); + numOpenIterators.increment(); + openIterators.add(this); } + @Override + public long startTimestamp() { + return startTimestampMs; + } @Override public void close() { try { iterator.close(); } finally { - numOpenIterators.decrementAndGet(); + sensor.record(time.nanoseconds() - startNs); + numOpenIterators.decrement(); + openIterators.remove(this); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java index 4bcbb483a314e..731bc3145c181 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java @@ -45,9 +45,12 @@ import org.apache.kafka.streams.state.internals.StoreQueryUtils.QueryHandler; import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics; +import java.util.Comparator; import java.util.Map; +import java.util.NavigableSet; import java.util.Objects; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.LongAdder; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -68,10 +71,12 @@ public class MeteredSessionStore private Sensor flushSensor; private Sensor removeSensor; private Sensor e2eLatencySensor; + private Sensor iteratorDurationSensor; private InternalProcessorContext context; private TaskId taskId; - private AtomicInteger numOpenIterators = new AtomicInteger(0); + private LongAdder numOpenIterators = new LongAdder(); + private final NavigableSet openIterators = new ConcurrentSkipListSet<>(Comparator.comparingLong(MeteredIterator::startTimestamp)); @SuppressWarnings("rawtypes") private final Map queryHandlers = @@ -134,8 +139,12 @@ private void registerMetrics() { flushSensor = StateStoreMetrics.flushSensor(taskId.toString(), metricsScope, name(), streamsMetrics); removeSensor = StateStoreMetrics.removeSensor(taskId.toString(), metricsScope, name(), streamsMetrics); e2eLatencySensor = StateStoreMetrics.e2ELatencySensor(taskId.toString(), metricsScope, name(), streamsMetrics); + iteratorDurationSensor = StateStoreMetrics.iteratorDurationSensor(taskId.toString(), metricsScope, name(), streamsMetrics); StateStoreMetrics.addNumOpenIteratorsGauge(taskId.toString(), metricsScope, name(), streamsMetrics, - (config, now) -> numOpenIterators.get()); + (config, now) -> numOpenIterators.sum()); + StateStoreMetrics.addOldestOpenIteratorGauge(taskId.toString(), metricsScope, name(), streamsMetrics, + (config, now) -> openIterators.isEmpty() ? null : openIterators.first().startTimestamp() + ); } @@ -250,11 +259,13 @@ public KeyValueIterator, V> fetch(final K key) { return new MeteredWindowedKeyValueIterator<>( wrapped().fetch(keyBytes(key)), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators); + numOpenIterators, + openIterators); } @Override @@ -263,11 +274,13 @@ public KeyValueIterator, V> backwardFetch(final K key) { return new MeteredWindowedKeyValueIterator<>( wrapped().backwardFetch(keyBytes(key)), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators + numOpenIterators, + openIterators ); } @@ -277,11 +290,13 @@ public KeyValueIterator, V> fetch(final K keyFrom, return new MeteredWindowedKeyValueIterator<>( wrapped().fetch(keyBytes(keyFrom), keyBytes(keyTo)), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators); + numOpenIterators, + openIterators); } @Override @@ -290,11 +305,13 @@ public KeyValueIterator, V> backwardFetch(final K keyFrom, return new MeteredWindowedKeyValueIterator<>( wrapped().backwardFetch(keyBytes(keyFrom), keyBytes(keyTo)), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators + numOpenIterators, + openIterators ); } @@ -310,11 +327,13 @@ public KeyValueIterator, V> findSessions(final K key, earliestSessionEndTime, latestSessionStartTime), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators); + numOpenIterators, + openIterators); } @Override @@ -330,11 +349,13 @@ public KeyValueIterator, V> backwardFindSessions(final K key, latestSessionStartTime ), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators + numOpenIterators, + openIterators ); } @@ -352,11 +373,13 @@ public KeyValueIterator, V> findSessions(final K keyFrom, earliestSessionEndTime, latestSessionStartTime), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators); + numOpenIterators, + openIterators); } @Override @@ -365,11 +388,13 @@ public KeyValueIterator, V> findSessions(final long earliestSessionE return new MeteredWindowedKeyValueIterator<>( wrapped().findSessions(earliestSessionEndTime, latestSessionEndTime), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators); + numOpenIterators, + openIterators); } @Override @@ -387,11 +412,13 @@ public KeyValueIterator, V> backwardFindSessions(final K keyFrom, latestSessionStartTime ), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators + numOpenIterators, + openIterators ); } @@ -458,11 +485,13 @@ private QueryResult runRangeQuery(final Query query, new MeteredWindowedKeyValueIterator<>( rawResult.getResult(), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, StoreQueryUtils.getDeserializeValue(serdes, wrapped()), time, - numOpenIterators + numOpenIterators, + openIterators ); final QueryResult> typedQueryResult = InternalQueryResultUtil.copyAndSubstituteDeserializedResult(rawResult, typedResult); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java index d0fcb0cf0ed4b..0b4702b9dbea5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java @@ -307,11 +307,12 @@ private QueryResult runRangeQuery(final Query query, } @SuppressWarnings("unchecked") - private class MeteredTimestampedKeyValueStoreIterator implements KeyValueIterator { + private class MeteredTimestampedKeyValueStoreIterator implements KeyValueIterator, MeteredIterator { private final KeyValueIterator iter; private final Sensor sensor; private final long startNs; + private final long startTimestampMs; private final Function> valueAndTimestampDeserializer; private final boolean returnPlainValue; @@ -324,8 +325,15 @@ private MeteredTimestampedKeyValueStoreIterator(final KeyValueIterator QueryResult runMultiVersionedKeyQuery(final Query query, final final QueryResult> rawResult = wrapped().query(rawKeyQuery, positionBound, config); if (rawResult.isSuccess()) { final MeteredMultiVersionedKeyQueryIterator typedResult = - new MeteredMultiVersionedKeyQueryIterator(rawResult.getResult(), StoreQueryUtils.getDeserializeValue(plainValueSerdes), numOpenIterators); + new MeteredMultiVersionedKeyQueryIterator( + rawResult.getResult(), + iteratorDurationSensor, + time, + StoreQueryUtils.getDeserializeValue(plainValueSerdes), + numOpenIterators, + openIterators + ); final QueryResult> typedQueryResult = InternalQueryResultUtil.copyAndSubstituteDeserializedResult(rawResult, typedResult); result = (QueryResult) typedQueryResult; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index 2d63e3ca7c5b3..a62e8c47563f2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -47,9 +47,12 @@ import org.apache.kafka.streams.state.internals.StoreQueryUtils.QueryHandler; import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics; +import java.util.Comparator; import java.util.Map; +import java.util.NavigableSet; import java.util.Objects; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.LongAdder; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -72,10 +75,12 @@ public class MeteredWindowStore private Sensor fetchSensor; private Sensor flushSensor; private Sensor e2eLatencySensor; + private Sensor iteratorDurationSensor; private InternalProcessorContext context; private TaskId taskId; - private AtomicInteger numOpenIterators = new AtomicInteger(0); + private LongAdder numOpenIterators = new LongAdder(); + private NavigableSet openIterators = new ConcurrentSkipListSet<>(Comparator.comparingLong(MeteredIterator::startTimestamp)); @SuppressWarnings("rawtypes") private final Map queryHandlers = @@ -153,8 +158,12 @@ private void registerMetrics() { fetchSensor = StateStoreMetrics.fetchSensor(taskId.toString(), metricsScope, name(), streamsMetrics); flushSensor = StateStoreMetrics.flushSensor(taskId.toString(), metricsScope, name(), streamsMetrics); e2eLatencySensor = StateStoreMetrics.e2ELatencySensor(taskId.toString(), metricsScope, name(), streamsMetrics); + iteratorDurationSensor = StateStoreMetrics.iteratorDurationSensor(taskId.toString(), metricsScope, name(), streamsMetrics); StateStoreMetrics.addNumOpenIteratorsGauge(taskId.toString(), metricsScope, name(), streamsMetrics, - (config, now) -> numOpenIterators.get()); + (config, now) -> numOpenIterators.sum()); + StateStoreMetrics.addOldestOpenIteratorGauge(taskId.toString(), metricsScope, name(), streamsMetrics, + (config, now) -> openIterators.isEmpty() ? null : openIterators.first().startTimestamp() + ); } @Deprecated @@ -239,10 +248,12 @@ public WindowStoreIterator fetch(final K key, return new MeteredWindowStoreIterator<>( wrapped().fetch(keyBytes(key), timeFrom, timeTo), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::valueFrom, time, - numOpenIterators + numOpenIterators, + openIterators ); } @@ -254,10 +265,12 @@ public WindowStoreIterator backwardFetch(final K key, return new MeteredWindowStoreIterator<>( wrapped().backwardFetch(keyBytes(key), timeFrom, timeTo), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::valueFrom, time, - numOpenIterators + numOpenIterators, + openIterators ); } @@ -273,11 +286,13 @@ public KeyValueIterator, V> fetch(final K keyFrom, timeFrom, timeTo), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators); + numOpenIterators, + openIterators); } @Override @@ -292,11 +307,13 @@ public KeyValueIterator, V> backwardFetch(final K keyFrom, timeFrom, timeTo), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators); + numOpenIterators, + openIterators); } @Override @@ -305,11 +322,13 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, return new MeteredWindowedKeyValueIterator<>( wrapped().fetchAll(timeFrom, timeTo), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators); + numOpenIterators, + openIterators); } @Override @@ -318,11 +337,13 @@ public KeyValueIterator, V> backwardFetchAll(final long timeFrom, return new MeteredWindowedKeyValueIterator<>( wrapped().backwardFetchAll(timeFrom, timeTo), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators); + numOpenIterators, + openIterators); } @Override @@ -330,11 +351,13 @@ public KeyValueIterator, V> all() { return new MeteredWindowedKeyValueIterator<>( wrapped().all(), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators + numOpenIterators, + openIterators ); } @@ -343,11 +366,13 @@ public KeyValueIterator, V> backwardAll() { return new MeteredWindowedKeyValueIterator<>( wrapped().backwardAll(), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, serdes::valueFrom, time, - numOpenIterators + numOpenIterators, + openIterators ); } @@ -420,11 +445,13 @@ private QueryResult runRangeQuery(final Query query, new MeteredWindowedKeyValueIterator<>( rawResult.getResult(), fetchSensor, + iteratorDurationSensor, streamsMetrics, serdes::keyFrom, getDeserializeValue(serdes, wrapped()), time, - numOpenIterators + numOpenIterators, + openIterators ); final QueryResult> typedQueryResult = InternalQueryResultUtil.copyAndSubstituteDeserializedResult(rawResult, typedResult); @@ -471,10 +498,12 @@ private QueryResult runKeyQuery(final Query query, final MeteredWindowStoreIterator typedResult = new MeteredWindowStoreIterator<>( rawResult.getResult(), fetchSensor, + iteratorDurationSensor, streamsMetrics, getDeserializeValue(serdes, wrapped()), time, - numOpenIterators + numOpenIterators, + openIterators ); final QueryResult> typedQueryResult = InternalQueryResultUtil.copyAndSubstituteDeserializedResult(rawResult, typedResult); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreIterator.java index 7f5cf99c0257b..90bd3e5f96f67 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreIterator.java @@ -22,33 +22,48 @@ import org.apache.kafka.streams.StreamsMetrics; import org.apache.kafka.streams.state.WindowStoreIterator; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.LongAdder; +import java.util.Set; import java.util.function.Function; -class MeteredWindowStoreIterator implements WindowStoreIterator { +class MeteredWindowStoreIterator implements WindowStoreIterator, MeteredIterator { private final WindowStoreIterator iter; - private final Sensor sensor; + private final Sensor operationSensor; + private final Sensor iteratorSensor; private final StreamsMetrics metrics; private final Function valueFrom; private final long startNs; + private final long startTimestampMs; private final Time time; - private final AtomicInteger numOpenIterators; + private final LongAdder numOpenIterators; + private final Set openIterators; MeteredWindowStoreIterator(final WindowStoreIterator iter, - final Sensor sensor, + final Sensor operationSensor, + final Sensor iteratorSensor, final StreamsMetrics metrics, final Function valueFrom, final Time time, - final AtomicInteger numOpenIterators) { + final LongAdder numOpenIterators, + final Set openIterators) { this.iter = iter; - this.sensor = sensor; + this.operationSensor = operationSensor; + this.iteratorSensor = iteratorSensor; this.metrics = metrics; this.valueFrom = valueFrom; this.startNs = time.nanoseconds(); + this.startTimestampMs = time.milliseconds(); this.time = time; this.numOpenIterators = numOpenIterators; - numOpenIterators.incrementAndGet(); + this.openIterators = openIterators; + numOpenIterators.increment(); + openIterators.add(this); + } + + @Override + public long startTimestamp() { + return startTimestampMs; } @Override @@ -67,8 +82,11 @@ public void close() { try { iter.close(); } finally { - sensor.record(time.nanoseconds() - startNs); - numOpenIterators.decrementAndGet(); + final long duration = time.nanoseconds() - startNs; + operationSensor.record(duration); + iteratorSensor.record(duration); + numOpenIterators.decrement(); + openIterators.remove(this); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowedKeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowedKeyValueIterator.java index a9354c863c81c..1eeacd81babc0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowedKeyValueIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowedKeyValueIterator.java @@ -24,36 +24,51 @@ import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.state.KeyValueIterator; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.LongAdder; +import java.util.Set; import java.util.function.Function; -class MeteredWindowedKeyValueIterator implements KeyValueIterator, V> { +class MeteredWindowedKeyValueIterator implements KeyValueIterator, V>, MeteredIterator { private final KeyValueIterator, byte[]> iter; - private final Sensor sensor; + private final Sensor operationSensor; + private final Sensor iteratorSensor; private final StreamsMetrics metrics; private final Function deserializeKey; private final Function deserializeValue; private final long startNs; + private final long startTimestampMs; private final Time time; - private final AtomicInteger numOpenIterators; + private final LongAdder numOpenIterators; + private final Set openIterators; MeteredWindowedKeyValueIterator(final KeyValueIterator, byte[]> iter, - final Sensor sensor, + final Sensor operationSensor, + final Sensor iteratorSensor, final StreamsMetrics metrics, final Function deserializeKey, final Function deserializeValue, final Time time, - final AtomicInteger numOpenIterators) { + final LongAdder numOpenIterators, + final Set openIterators) { this.iter = iter; - this.sensor = sensor; + this.operationSensor = operationSensor; + this.iteratorSensor = iteratorSensor; this.metrics = metrics; this.deserializeKey = deserializeKey; this.deserializeValue = deserializeValue; this.startNs = time.nanoseconds(); + this.startTimestampMs = time.milliseconds(); this.time = time; this.numOpenIterators = numOpenIterators; - numOpenIterators.incrementAndGet(); + this.openIterators = openIterators; + numOpenIterators.increment(); + openIterators.add(this); + } + + @Override + public long startTimestamp() { + return this.startTimestampMs; } @Override @@ -77,8 +92,11 @@ public void close() { try { iter.close(); } finally { - sensor.record(time.nanoseconds() - startNs); - numOpenIterators.decrementAndGet(); + final long duration = time.nanoseconds() - startNs; + operationSensor.record(duration); + iteratorSensor.record(duration); + numOpenIterators.decrement(); + openIterators.remove(this); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedKeyAndJoinSide.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedKeyAndJoinSide.java index c0516e101540e..3b799e815399f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedKeyAndJoinSide.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedKeyAndJoinSide.java @@ -42,17 +42,25 @@ private TimestampedKeyAndJoinSide(final boolean leftSide, final K key, final lon } /** - * Create a new {@link TimestampedKeyAndJoinSide} instance if the provide {@code key} is not {@code null}. + * Create a new {@link TimestampedKeyAndJoinSide} instance for the left join side if the provide {@code key} is not {@code null}. * - * @param leftSide True if the key is part of the left join side; False if it is from the right join side * @param key the key * @param the type of the key - * @return a new {@link TimestampedKeyAndJoinSide} instance if the provide {@code key} is not {@code null} + * @return a new {@link TimestampedKeyAndJoinSide} instance for the left join side if the provide {@code key} is not {@code null} */ - public static TimestampedKeyAndJoinSide make(final boolean leftSide, final K key, final long timestamp) { - return new TimestampedKeyAndJoinSide<>(leftSide, key, timestamp); + public static TimestampedKeyAndJoinSide makeLeft(final K key, final long timestamp) { + return new TimestampedKeyAndJoinSide<>(true, key, timestamp); + } + /** + * Create a new {@link TimestampedKeyAndJoinSide} instance for the right join side if the provide {@code key} is not {@code null}. + * + * @param key the key + * @param the type of the key + * @return a new {@link TimestampedKeyAndJoinSide} instance for the right join side if the provide {@code key} is not {@code null} + */ + public static TimestampedKeyAndJoinSide makeRight(final K key, final long timestamp) { + return new TimestampedKeyAndJoinSide<>(false, key, timestamp); } - public boolean isLeftSide() { return leftSide; } @@ -89,4 +97,4 @@ public boolean equals(final Object o) { public int hashCode() { return Objects.hash(leftSide, key, timestamp); } -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedKeyAndJoinSideDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedKeyAndJoinSideDeserializer.java index 9ecea46c84e78..4b035eff6d873 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedKeyAndJoinSideDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedKeyAndJoinSideDeserializer.java @@ -55,11 +55,12 @@ public void configure(final Map configs, final boolean isKey) { @Override public TimestampedKeyAndJoinSide deserialize(final String topic, final byte[] data) { - final boolean bool = data[StateSerdes.TIMESTAMP_SIZE] == 1; + final boolean isLeft = data[StateSerdes.TIMESTAMP_SIZE] == 1; final K key = keyDeserializer.deserialize(topic, rawKey(data)); final long timestamp = timestampDeserializer.deserialize(topic, rawTimestamp(data)); - return TimestampedKeyAndJoinSide.make(bool, key, timestamp); + return isLeft ? TimestampedKeyAndJoinSide.makeLeft(key, timestamp) : + TimestampedKeyAndJoinSide.makeRight(key, timestamp); } private byte[] rawTimestamp(final byte[] data) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/StateStoreMetrics.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/StateStoreMetrics.java index eea7a9644221c..2422fa9d5e321 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/StateStoreMetrics.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/StateStoreMetrics.java @@ -149,6 +149,18 @@ private StateStoreMetrics() {} private static final String NUM_OPEN_ITERATORS_DESCRIPTION = "The current number of iterators on the store that have been created, but not yet closed"; + private static final String ITERATOR_DURATION = "iterator-duration"; + private static final String ITERATOR_DURATION_DESCRIPTION = + "time spent between creating an iterator and closing it, in nanoseconds"; + private static final String ITERATOR_DURATION_AVG_DESCRIPTION = + AVG_DESCRIPTION_PREFIX + ITERATOR_DURATION_DESCRIPTION; + private static final String ITERATOR_DURATION_MAX_DESCRIPTION = + MAX_DESCRIPTION_PREFIX + ITERATOR_DURATION_DESCRIPTION; + + private static final String OLDEST_ITERATOR_OPEN_SINCE_MS = "oldest-iterator-open-since-ms"; + private static final String OLDEST_ITERATOR_OPEN_SINCE_MS_DESCRIPTION = + "The UNIX timestamp the oldest still open iterator was created, in milliseconds"; + public static Sensor putSensor(final String taskId, final String storeType, final String storeName, @@ -409,11 +421,28 @@ public static Sensor e2ELatencySensor(final String taskId, return sensor; } + public static Sensor iteratorDurationSensor(final String taskId, + final String storeType, + final String storeName, + final StreamsMetricsImpl streamsMetrics) { + final Sensor sensor = streamsMetrics.storeLevelSensor(taskId, storeName, ITERATOR_DURATION, RecordingLevel.DEBUG); + final Map tagMap = streamsMetrics.storeLevelTagMap(taskId, storeType, storeName); + addAvgAndMaxToSensor( + sensor, + STATE_STORE_LEVEL_GROUP, + tagMap, + ITERATOR_DURATION, + ITERATOR_DURATION_AVG_DESCRIPTION, + ITERATOR_DURATION_MAX_DESCRIPTION + ); + return sensor; + } + public static void addNumOpenIteratorsGauge(final String taskId, final String storeType, final String storeName, final StreamsMetricsImpl streamsMetrics, - final Gauge numOpenIteratorsGauge) { + final Gauge numOpenIteratorsGauge) { streamsMetrics.addStoreLevelMutableMetric( taskId, storeType, @@ -426,6 +455,22 @@ public static void addNumOpenIteratorsGauge(final String taskId, } + public static void addOldestOpenIteratorGauge(final String taskId, + final String storeType, + final String storeName, + final StreamsMetricsImpl streamsMetrics, + final Gauge oldestOpenIteratorGauge) { + streamsMetrics.addStoreLevelMutableMetric( + taskId, + storeType, + storeName, + OLDEST_ITERATOR_OPEN_SINCE_MS, + OLDEST_ITERATOR_OPEN_SINCE_MS_DESCRIPTION, + RecordingLevel.INFO, + oldestOpenIteratorGauge + ); + } + private static Sensor sizeOrCountSensor(final String taskId, final String storeType, final String storeName, diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index a264de083b47f..8c89132ae2f9c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -67,6 +67,7 @@ import static org.apache.kafka.streams.StreamsConfig.RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG; import static org.apache.kafka.streams.StreamsConfig.RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG; import static org.apache.kafka.streams.StreamsConfig.STATE_DIR_CONFIG; +import static org.apache.kafka.streams.StreamsConfig.TASK_ASSIGNOR_CLASS_CONFIG; import static org.apache.kafka.streams.StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG; import static org.apache.kafka.streams.StreamsConfig.adminClientPrefix; import static org.apache.kafka.streams.StreamsConfig.consumerPrefix; @@ -1457,6 +1458,12 @@ public void shouldReturnRackAwareAssignmentNonOverlapCost() { assertEquals(Integer.valueOf(10), new StreamsConfig(props).getInt(RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG)); } + @Test + public void shouldReturnTaskAssignorClass() { + props.put(StreamsConfig.TASK_ASSIGNOR_CLASS_CONFIG, "StickyTaskAssignor"); + assertEquals("StickyTaskAssignor", new StreamsConfig(props).getString(TASK_ASSIGNOR_CLASS_CONFIG)); + } + @Test public void shouldReturnDefaultClientSupplier() { final KafkaClientSupplier supplier = streamsConfig.getKafkaClientSupplier(); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/HighAvailabilityTaskAssignorIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/HighAvailabilityTaskAssignorIntegrationTest.java index 7d512c14f57f9..1013d23413174 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/HighAvailabilityTaskAssignorIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/HighAvailabilityTaskAssignorIntegrationTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.integration; import java.util.stream.Stream; -import kafka.server.KafkaConfig; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -31,6 +30,7 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; @@ -85,13 +85,13 @@ public class HighAvailabilityTaskAssignorIntegrationTest { new Properties(), asList( new Properties() {{ - setProperty(KafkaConfig.RackProp(), AssignmentTestUtils.RACK_0); + setProperty(ServerConfigs.BROKER_RACK_CONFIG, AssignmentTestUtils.RACK_0); }}, new Properties() {{ - setProperty(KafkaConfig.RackProp(), AssignmentTestUtils.RACK_1); + setProperty(ServerConfigs.BROKER_RACK_CONFIG, AssignmentTestUtils.RACK_1); }}, new Properties() {{ - setProperty(KafkaConfig.RackProp(), AssignmentTestUtils.RACK_2); + setProperty(ServerConfigs.BROKER_RACK_CONFIG, AssignmentTestUtils.RACK_2); }} ) ); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java index 6ea41b1e76bb2..7bd9ce8164ec8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams.integration.utils; -import kafka.server.KafkaConfig; import kafka.server.KafkaServer; import kafka.zk.EmbeddedZookeeper; import org.apache.kafka.clients.admin.Admin; @@ -26,6 +25,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.apache.kafka.network.SocketServerConfigs; import org.apache.kafka.server.config.ConfigType; +import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.config.ZkConfigs; import org.apache.kafka.server.util.MockTime; @@ -116,7 +116,7 @@ public void start() throws IOException { brokerConfig.put(ZkConfigs.ZK_CONNECT_CONFIG, zKConnectString()); putIfAbsent(brokerConfig, SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:" + DEFAULT_BROKER_PORT); - putIfAbsent(brokerConfig, KafkaConfig.DeleteTopicEnableProp(), true); + putIfAbsent(brokerConfig, ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, true); putIfAbsent(brokerConfig, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, 2 * 1024 * 1024L); putIfAbsent(brokerConfig, GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, 0); putIfAbsent(brokerConfig, GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, 0); @@ -126,7 +126,7 @@ public void start() throws IOException { putIfAbsent(brokerConfig, ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, true); for (int i = 0; i < brokers.length; i++) { - brokerConfig.put(KafkaConfig.BrokerIdProp(), i); + brokerConfig.put(ServerConfigs.BROKER_ID_CONFIG, i); log.debug("Starting a Kafka instance on {} ...", brokerConfig.get(SocketServerConfigs.LISTENERS_CONFIG)); final Properties effectiveConfig = new Properties(); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java index b1aca31563218..87defb89aaa6c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java @@ -29,6 +29,7 @@ import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.network.SocketServerConfigs; +import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.config.ZkConfigs; import org.apache.kafka.server.util.MockTime; import org.slf4j.Logger; @@ -92,11 +93,11 @@ public KafkaEmbedded(final Properties config, final MockTime time) throws IOExce */ private Properties effectiveConfigFrom(final Properties initialConfig) { final Properties effectiveConfig = new Properties(); - effectiveConfig.put(KafkaConfig.BrokerIdProp(), 0); + effectiveConfig.put(ServerConfigs.BROKER_ID_CONFIG, 0); effectiveConfig.put(NUM_PARTITIONS_CONFIG, 1); effectiveConfig.put(AUTO_CREATE_TOPICS_ENABLE_CONFIG, true); - effectiveConfig.put(KafkaConfig.MessageMaxBytesProp(), 1000000); - effectiveConfig.put(KafkaConfig.ControlledShutdownEnableProp(), true); + effectiveConfig.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, 1000000); + effectiveConfig.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true); effectiveConfig.put(ZkConfigs.ZK_SESSION_TIMEOUT_MS_CONFIG, 10000); effectiveConfig.putAll(initialConfig); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index 7fd62bb78bddd..3c602ee8d5d7e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -456,8 +456,7 @@ public void shouldThrottleEmitNonJoinedOuterRecordsEvenWhenClockDrift() { * This test is testing something internal to [[KStreamKStreamJoin]], so we had to setup low-level api manually. */ final KStreamImplJoin.TimeTrackerSupplier tracker = new KStreamImplJoin.TimeTrackerSupplier(); - final KStreamKStreamJoin join = new KStreamKStreamJoin<>( - false, + final KStreamKStreamJoinRightSide join = new KStreamKStreamJoinRightSide<>( "other", new JoinWindowsInternal(JoinWindows.ofTimeDifferenceWithNoGrace(ofMillis(1000))), (key, v1, v2) -> v1 + v2, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java index 00151c6798105..860be7f7efe04 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java @@ -30,7 +30,6 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.ProcessorContext; @@ -51,6 +50,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import static org.apache.kafka.common.utils.Utils.mkEntry; @@ -98,10 +98,12 @@ public class MeteredKeyValueStoreTest { private MeteredKeyValueStore metered; private final Metrics metrics = new Metrics(); private Map tags; + private MockTime mockTime; @Before public void before() { - final Time mockTime = new MockTime(); + final MockTime mockTime = new MockTime(); + this.mockTime = mockTime; metered = new MeteredKeyValueStore<>( inner, STORE_TYPE, @@ -449,13 +451,79 @@ public void shouldTrackOpenIteratorsMetric() { final KafkaMetric openIteratorsMetric = metric("num-open-iterators"); assertThat(openIteratorsMetric, not(nullValue())); - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L)); try (final KeyValueIterator iterator = metered.prefixScan(KEY, stringSerializer)) { - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(1)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(1L)); } - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L)); + } + + @Test + public void shouldTimeIteratorDuration() { + when(inner.all()).thenReturn(KeyValueIterators.emptyIterator()); + init(); + + final KafkaMetric iteratorDurationAvgMetric = metric("iterator-duration-avg"); + final KafkaMetric iteratorDurationMaxMetric = metric("iterator-duration-max"); + assertThat(iteratorDurationAvgMetric, not(nullValue())); + assertThat(iteratorDurationMaxMetric, not(nullValue())); + + assertThat((Double) iteratorDurationAvgMetric.metricValue(), equalTo(Double.NaN)); + assertThat((Double) iteratorDurationMaxMetric.metricValue(), equalTo(Double.NaN)); + + try (final KeyValueIterator iterator = metered.all()) { + // nothing to do, just close immediately + mockTime.sleep(2); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + + try (final KeyValueIterator iterator = metered.all()) { + // nothing to do, just close immediately + mockTime.sleep(3); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.5 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(3.0 * TimeUnit.MILLISECONDS.toNanos(1))); + } + + @Test + public void shouldTrackOldestOpenIteratorTimestamp() { + when(inner.all()).thenReturn(KeyValueIterators.emptyIterator()); + init(); + + final KafkaMetric oldestIteratorTimestampMetric = metric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, not(nullValue())); + + assertThat(oldestIteratorTimestampMetric.metricValue(), nullValue()); + + KeyValueIterator second = null; + final long secondTimestamp; + try { + try (final KeyValueIterator first = metered.all()) { + final long oldestTimestamp = mockTime.milliseconds(); + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(oldestTimestamp)); + mockTime.sleep(100); + + // open a second iterator before closing the first to test that we still produce the first iterator's timestamp + second = metered.all(); + secondTimestamp = mockTime.milliseconds(); + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(oldestTimestamp)); + mockTime.sleep(100); + } + + // now that the first iterator is closed, check that the timestamp has advanced to the still open second iterator + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(secondTimestamp)); + } finally { + if (second != null) { + second.close(); + } + } + + assertThat((Integer) oldestIteratorTimestampMetric.metricValue(), nullValue()); } private KafkaMetric metric(final MetricName metricName) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java index b7c99032f8920..68ec8f1c79832 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java @@ -53,6 +53,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import static org.apache.kafka.common.utils.Utils.mkEntry; @@ -95,6 +96,7 @@ public class MeteredSessionStoreTest { private final String threadId = Thread.currentThread().getName(); private final TaskId taskId = new TaskId(0, 0, "My-Topology"); private final Metrics metrics = new Metrics(); + private MockTime mockTime; private MeteredSessionStore store; @Mock private SessionStore innerStore; @@ -105,7 +107,7 @@ public class MeteredSessionStoreTest { @Before public void before() { - final Time mockTime = new MockTime(); + mockTime = new MockTime(); store = new MeteredSessionStore<>( innerStore, STORE_TYPE, @@ -613,13 +615,79 @@ public void shouldTrackOpenIteratorsMetric() { final KafkaMetric openIteratorsMetric = metric("num-open-iterators"); assertThat(openIteratorsMetric, not(nullValue())); - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L)); try (final KeyValueIterator, String> iterator = store.backwardFetch(KEY)) { - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(1)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(1L)); } - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L)); + } + + @Test + public void shouldTimeIteratorDuration() { + when(innerStore.backwardFetch(KEY_BYTES)).thenReturn(KeyValueIterators.emptyIterator()); + init(); + + final KafkaMetric iteratorDurationAvgMetric = metric("iterator-duration-avg"); + final KafkaMetric iteratorDurationMaxMetric = metric("iterator-duration-max"); + assertThat(iteratorDurationAvgMetric, not(nullValue())); + assertThat(iteratorDurationMaxMetric, not(nullValue())); + + assertThat((Double) iteratorDurationAvgMetric.metricValue(), equalTo(Double.NaN)); + assertThat((Double) iteratorDurationMaxMetric.metricValue(), equalTo(Double.NaN)); + + try (final KeyValueIterator, String> iterator = store.backwardFetch(KEY)) { + // nothing to do, just close immediately + mockTime.sleep(2); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + + try (final KeyValueIterator, String> iterator = store.backwardFetch(KEY)) { + // nothing to do, just close immediately + mockTime.sleep(3); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.5 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(3.0 * TimeUnit.MILLISECONDS.toNanos(1))); + } + + @Test + public void shouldTrackOldestOpenIteratorTimestamp() { + when(innerStore.backwardFetch(KEY_BYTES)).thenReturn(KeyValueIterators.emptyIterator()); + init(); + + final KafkaMetric oldestIteratorTimestampMetric = metric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, not(nullValue())); + + assertThat(oldestIteratorTimestampMetric.metricValue(), nullValue()); + + KeyValueIterator, String> second = null; + final long secondTimestamp; + try { + try (final KeyValueIterator, String> first = store.backwardFetch(KEY)) { + final long oldestTimestamp = mockTime.milliseconds(); + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(oldestTimestamp)); + mockTime.sleep(100); + + // open a second iterator before closing the first to test that we still produce the first iterator's timestamp + second = store.backwardFetch(KEY); + secondTimestamp = mockTime.milliseconds(); + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(oldestTimestamp)); + mockTime.sleep(100); + } + + // now that the first iterator is closed, check that the timestamp has advanced to the still open second iterator + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(secondTimestamp)); + } finally { + if (second != null) { + second.close(); + } + } + + assertThat((Integer) oldestIteratorTimestampMetric.metricValue(), nullValue()); } private KafkaMetric metric(final String name) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java index f629acfc9a8a6..32d88e8c8ce0d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java @@ -29,7 +29,6 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; @@ -53,6 +52,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -95,6 +95,7 @@ public class MeteredTimestampedKeyValueStoreTest { private KeyValueStore inner; @Mock private InternalProcessorContext context; + private MockTime mockTime; private final static Map CONFIGS = mkMap(mkEntry(StreamsConfig.InternalConfig.TOPIC_PREFIX_ALTERNATIVE, APPLICATION_ID)); @@ -107,7 +108,7 @@ public class MeteredTimestampedKeyValueStoreTest { @Before public void before() { - final Time mockTime = new MockTime(); + mockTime = new MockTime(); metered = new MeteredTimestampedKeyValueStore<>( inner, "scope", @@ -448,12 +449,78 @@ public void shouldTrackOpenIteratorsMetric() { final KafkaMetric openIteratorsMetric = metric("num-open-iterators"); assertThat(openIteratorsMetric, not(nullValue())); - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L)); try (final KeyValueIterator> iterator = metered.all()) { - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(1)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(1L)); } - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L)); + } + + @Test + public void shouldTimeIteratorDuration() { + when(inner.all()).thenReturn(KeyValueIterators.emptyIterator()); + init(); + + final KafkaMetric iteratorDurationAvgMetric = metric("iterator-duration-avg"); + final KafkaMetric iteratorDurationMaxMetric = metric("iterator-duration-max"); + assertThat(iteratorDurationAvgMetric, not(nullValue())); + assertThat(iteratorDurationMaxMetric, not(nullValue())); + + assertThat((Double) iteratorDurationAvgMetric.metricValue(), equalTo(Double.NaN)); + assertThat((Double) iteratorDurationMaxMetric.metricValue(), equalTo(Double.NaN)); + + try (final KeyValueIterator> iterator = metered.all()) { + // nothing to do, just close immediately + mockTime.sleep(2); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + + try (final KeyValueIterator> iterator = metered.all()) { + // nothing to do, just close immediately + mockTime.sleep(3); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.5 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(3.0 * TimeUnit.MILLISECONDS.toNanos(1))); + } + + @Test + public void shouldTrackOldestOpenIteratorTimestamp() { + when(inner.all()).thenReturn(KeyValueIterators.emptyIterator()); + init(); + + final KafkaMetric oldestIteratorTimestampMetric = metric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, not(nullValue())); + + assertThat(oldestIteratorTimestampMetric.metricValue(), nullValue()); + + KeyValueIterator> second = null; + final long secondTimestamp; + try { + try (final KeyValueIterator> first = metered.all()) { + final long oldestTimestamp = mockTime.milliseconds(); + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(oldestTimestamp)); + mockTime.sleep(100); + + // open a second iterator before closing the first to test that we still produce the first iterator's timestamp + second = metered.all(); + secondTimestamp = mockTime.milliseconds(); + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(oldestTimestamp)); + mockTime.sleep(100); + } + + // now that the first iterator is closed, check that the timestamp has advanced to the still open second iterator + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(secondTimestamp)); + } finally { + if (second != null) { + second.close(); + } + } + + assertThat((Integer) oldestIteratorTimestampMetric.metricValue(), nullValue()); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStoreTest.java index 99f76839041c7..1e2a765bd021e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStoreTest.java @@ -39,6 +39,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.KafkaMetric; @@ -379,15 +380,92 @@ public void shouldTrackOpenIteratorsMetric() { final KafkaMetric openIteratorsMetric = getMetric("num-open-iterators"); assertThat(openIteratorsMetric, not(nullValue())); - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L)); final QueryResult> result = store.query(query, bound, config); try (final VersionedRecordIterator iterator = result.getResult()) { - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(1)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(1L)); } - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L)); + } + + @Test + public void shouldTimeIteratorDuration() { + final MultiVersionedKeyQuery query = MultiVersionedKeyQuery.withKey(KEY); + final PositionBound bound = PositionBound.unbounded(); + final QueryConfig config = new QueryConfig(false); + when(inner.query(any(), any(), any())).thenReturn( + QueryResult.forResult(new LogicalSegmentIterator(Collections.emptyListIterator(), RAW_KEY, 0L, 0L, ResultOrder.ANY))); + + final KafkaMetric iteratorDurationAvgMetric = getMetric("iterator-duration-avg"); + final KafkaMetric iteratorDurationMaxMetric = getMetric("iterator-duration-max"); + assertThat(iteratorDurationAvgMetric, not(nullValue())); + assertThat(iteratorDurationMaxMetric, not(nullValue())); + + assertThat((Double) iteratorDurationAvgMetric.metricValue(), equalTo(Double.NaN)); + assertThat((Double) iteratorDurationMaxMetric.metricValue(), equalTo(Double.NaN)); + + final QueryResult> first = store.query(query, bound, config); + try (final VersionedRecordIterator iterator = first.getResult()) { + // nothing to do, just close immediately + mockTime.sleep(2); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + + final QueryResult> second = store.query(query, bound, config); + try (final VersionedRecordIterator iterator = second.getResult()) { + // nothing to do, just close immediately + mockTime.sleep(3); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.5 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(3.0 * TimeUnit.MILLISECONDS.toNanos(1))); + } + + @Test + public void shouldTrackOldestOpenIteratorTimestamp() { + final MultiVersionedKeyQuery query = MultiVersionedKeyQuery.withKey(KEY); + final PositionBound bound = PositionBound.unbounded(); + final QueryConfig config = new QueryConfig(false); + when(inner.query(any(), any(), any())).thenReturn( + QueryResult.forResult(new LogicalSegmentIterator(Collections.emptyListIterator(), RAW_KEY, 0L, 0L, ResultOrder.ANY))); + + final KafkaMetric oldestIteratorTimestampMetric = getMetric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, not(nullValue())); + + assertThat(oldestIteratorTimestampMetric.metricValue(), nullValue()); + + final QueryResult> first = store.query(query, bound, config); + VersionedRecordIterator secondIterator = null; + final long secondTime; + try { + try (final VersionedRecordIterator iterator = first.getResult()) { + final long oldestTimestamp = mockTime.milliseconds(); + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(oldestTimestamp)); + mockTime.sleep(100); + + // open a second iterator before closing the first to test that we still produce the first iterator's timestamp + final QueryResult> second = store.query(query, bound, config); + secondIterator = second.getResult(); + secondTime = mockTime.milliseconds(); + + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(oldestTimestamp)); + mockTime.sleep(100); + } + + // now that the first iterator is closed, check that the timestamp has advanced to the still open second iterator + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(secondTime)); + } finally { + if (secondIterator != null) { + secondIterator.close(); + } + } + + assertThat((Integer) oldestIteratorTimestampMetric.metricValue(), nullValue()); } private KafkaMetric getMetric(final String name) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java index 3997a5e549ec1..46e7d6f69cd54 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java @@ -52,6 +52,7 @@ import java.time.temporal.ChronoUnit; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import static java.time.Instant.ofEpochMilli; @@ -94,11 +95,12 @@ public class MeteredWindowStoreTest { private InternalMockProcessorContext context; @SuppressWarnings("unchecked") private final WindowStore innerStoreMock = mock(WindowStore.class); + private final MockTime mockTime = new MockTime(); private MeteredWindowStore store = new MeteredWindowStore<>( innerStoreMock, WINDOW_SIZE_MS, // any size STORE_TYPE, - new MockTime(), + mockTime, Serdes.String(), new SerdeThatDoesntHandleNull() ); @@ -454,13 +456,79 @@ public void shouldTrackOpenIteratorsMetric() { final KafkaMetric openIteratorsMetric = metric("num-open-iterators"); assertThat(openIteratorsMetric, not(nullValue())); - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L)); try (final KeyValueIterator, String> iterator = store.all()) { - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(1)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(1L)); } - assertThat((Integer) openIteratorsMetric.metricValue(), equalTo(0)); + assertThat((Long) openIteratorsMetric.metricValue(), equalTo(0L)); + } + + @Test + public void shouldTimeIteratorDuration() { + when(innerStoreMock.all()).thenReturn(KeyValueIterators.emptyIterator()); + store.init((StateStoreContext) context, store); + + final KafkaMetric iteratorDurationAvgMetric = metric("iterator-duration-avg"); + final KafkaMetric iteratorDurationMaxMetric = metric("iterator-duration-max"); + assertThat(iteratorDurationAvgMetric, not(nullValue())); + assertThat(iteratorDurationMaxMetric, not(nullValue())); + + assertThat((Double) iteratorDurationAvgMetric.metricValue(), equalTo(Double.NaN)); + assertThat((Double) iteratorDurationMaxMetric.metricValue(), equalTo(Double.NaN)); + + try (final KeyValueIterator, String> iterator = store.all()) { + // nothing to do, just close immediately + mockTime.sleep(2); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(2.0 * TimeUnit.MILLISECONDS.toNanos(1))); + + try (final KeyValueIterator, String> iterator = store.all()) { + // nothing to do, just close immediately + mockTime.sleep(3); + } + + assertThat((double) iteratorDurationAvgMetric.metricValue(), equalTo(2.5 * TimeUnit.MILLISECONDS.toNanos(1))); + assertThat((double) iteratorDurationMaxMetric.metricValue(), equalTo(3.0 * TimeUnit.MILLISECONDS.toNanos(1))); + } + + @Test + public void shouldTrackOldestOpenIteratorTimestamp() { + when(innerStoreMock.all()).thenReturn(KeyValueIterators.emptyIterator()); + store.init((StateStoreContext) context, store); + + final KafkaMetric oldestIteratorTimestampMetric = metric("oldest-iterator-open-since-ms"); + assertThat(oldestIteratorTimestampMetric, not(nullValue())); + + assertThat(oldestIteratorTimestampMetric.metricValue(), nullValue()); + + KeyValueIterator, String> second = null; + final long secondTimestamp; + try { + try (final KeyValueIterator, String> first = store.all()) { + final long oldestTimestamp = mockTime.milliseconds(); + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(oldestTimestamp)); + mockTime.sleep(100); + + // open a second iterator before closing the first to test that we still produce the first iterator's timestamp + second = store.all(); + secondTimestamp = mockTime.milliseconds(); + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(oldestTimestamp)); + mockTime.sleep(100); + } + + // now that the first iterator is closed, check that the timestamp has advanced to the still open second iterator + assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(secondTimestamp)); + } finally { + if (second != null) { + second.close(); + } + } + + assertThat((Integer) oldestIteratorTimestampMetric.metricValue(), nullValue()); } private KafkaMetric metric(final String name) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedKeyAndJoinSideSerializerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedKeyAndJoinSideSerializerTest.java index 5cca8f6ba54da..8f4191fb8c6a0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedKeyAndJoinSideSerializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedKeyAndJoinSideSerializerTest.java @@ -34,7 +34,7 @@ public class TimestampedKeyAndJoinSideSerializerTest { public void shouldSerializeKeyWithJoinSideAsTrue() { final String value = "some-string"; - final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide = TimestampedKeyAndJoinSide.make(true, value, 10); + final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide = TimestampedKeyAndJoinSide.makeLeft(value, 10); final byte[] serialized = STRING_SERDE.serializer().serialize(TOPIC, timestampedKeyAndJoinSide); @@ -51,7 +51,7 @@ public void shouldSerializeKeyWithJoinSideAsTrue() { public void shouldSerializeKeyWithJoinSideAsFalse() { final String value = "some-string"; - final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide = TimestampedKeyAndJoinSide.make(false, value, 20); + final TimestampedKeyAndJoinSide timestampedKeyAndJoinSide = TimestampedKeyAndJoinSide.makeRight(value, 20); final byte[] serialized = STRING_SERDE.serializer().serialize(TOPIC, timestampedKeyAndJoinSide); @@ -67,6 +67,6 @@ public void shouldSerializeKeyWithJoinSideAsFalse() { @Test public void shouldThrowIfSerializeNullData() { assertThrows(NullPointerException.class, - () -> STRING_SERDE.serializer().serialize(TOPIC, TimestampedKeyAndJoinSide.make(true, null, 0))); + () -> STRING_SERDE.serializer().serialize(TOPIC, TimestampedKeyAndJoinSide.makeLeft(null, 0))); } } diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index 25ccfb7c5e31f..9d6342a68ebc5 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -14,6 +14,34 @@ # limitations under the License. ARG jdk_version=openjdk:8 +FROM $jdk_version AS build-native-image + +WORKDIR /build + +COPY native/ native + +ARG KAFKA_MODE +ARG GRAALVM_URL="https://github.com/graalvm/graalvm-ce-builds/releases/download/jdk-21.0.1/graalvm-community-jdk-21.0.1_linux-aarch64_bin.tar.gz" + +ENV NATIVE_IMAGE_PATH="/build/graalvm/bin/native-image" +ENV NATIVE_CONFIGS_DIR="/build/native/native-image-configs" +ENV KAFKA_LIBS_DIR="/build/kafka/libs" +ENV KAFKA_BIN_DIR="/build/kafka-binary" +ENV TARGET_PATH="$KAFKA_BIN_DIR/kafka.Kafka" + +RUN mkdir $KAFKA_BIN_DIR + +RUN if [ "$KAFKA_MODE" = "native" ]; then \ + apt update && apt install -y sudo build-essential libz-dev zlib1g-dev curl jq coreutils libffi-dev cmake pkg-config libfuse-dev && apt-get -y clean ; \ + mkdir graalvm ; \ + curl -L "$GRAALVM_URL" -o graalvm.tar.gz ; \ + tar -xzf graalvm.tar.gz -C graalvm --strip-components=1 ; \ + mkdir kafka ; \ + tar xfz native/kafka.tgz -C kafka --strip-components 1 ; \ + rm graalvm.tar.gz kafka.tgz ; \ + /build/native/native_command.sh $NATIVE_IMAGE_PATH $NATIVE_CONFIGS_DIR $KAFKA_LIBS_DIR $TARGET_PATH ; \ + fi + FROM $jdk_version MAINTAINER Apache Kafka dev@kafka.apache.org @@ -37,6 +65,7 @@ RUN apt update && apt install -y sudo git netcat iptables rsync unzip wget curl RUN python3 -m pip install -U pip==21.1.1; RUN pip3 install --upgrade cffi virtualenv pyasn1 boto3 pycrypto pywinrm ipaddress enum34 debugpy && pip3 install --upgrade "ducktape>0.8" +COPY --from=build-native-image /build/kafka-binary/ /opt/kafka-binary/ # Set up ssh COPY ./ssh-config /root/.ssh/config # NOTE: The paramiko library supports the PEM-format private key, but does not support the RFC4716 format. @@ -107,7 +136,7 @@ ARG KIBOSH_VERSION="8841dd392e6fbf02986e2fb1f1ebf04df344b65a" ARG UID="1000" # Install Kibosh -RUN apt-get install fuse +RUN apt-get install fuse -y RUN cd /opt && git clone -q https://github.com/confluentinc/kibosh.git && cd "/opt/kibosh" && git reset --hard $KIBOSH_VERSION && mkdir "/opt/kibosh/build" && cd "/opt/kibosh/build" && ../configure && make -j 2 # Set up the ducker user. diff --git a/tests/docker/ducker-ak b/tests/docker/ducker-ak index 4add3172022fd..21f0f64219469 100755 --- a/tests/docker/ducker-ak +++ b/tests/docker/ducker-ak @@ -31,6 +31,9 @@ ducker_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" # The absolute path to the root Kafka directory kafka_dir="$( cd "${ducker_dir}/../.." && pwd )" +# The absolute path to the directory to hold the necessary files to construct a ducker image for executing Kafka in native mode. +tmp_native_dir=${ducker_dir}/native + # The memory consumption to allow during the docker build. # This does not include swap. docker_build_memory_limit="3200m" @@ -47,6 +50,9 @@ default_jdk="openjdk:8" # The default ducker-ak image name. default_image_name="ducker-ak" +# The default kafka server mode. +default_kafka_mode="jvm" + # Port to listen on when debugging debugpy_port=5678 @@ -247,6 +253,7 @@ ducker_build() { --build-arg "ducker_creator=${user_name}" \ --build-arg "jdk_version=${jdk_version}" \ --build-arg "UID=${UID}" \ + --build-arg "KAFKA_MODE=${kafka_mode}" \ -t "${image_name}" \ -f "${ducker_dir}/Dockerfile" ${docker_args} -- . docker_status=$? @@ -257,6 +264,7 @@ ducker_build() { $((${duration} % 60))s." echo "If this error is unexpected, consider running 'docker system prune -a' \ to clear old images from your local cache." + cleanup_native_dir exit 1 fi echo "Successfully built ${what} image in $((${duration} / 60))m \ @@ -305,6 +313,32 @@ setup_custom_ducktape() { must_do -v docker rm ducker01 } +cleanup_native_dir() { + if [ -d "${tmp_native_dir}" ]; then + echo "Deleting temporary native dir: ${tmp_native_dir}" + rm -rf "${tmp_native_dir}" + fi +} + +prepare_native_dir() { + echo "Mode provided for system tests run: $kafka_mode" + must_pushd "${kafka_dir}" + cleanup_native_dir && mkdir "${tmp_native_dir}" + + if [ "$kafka_mode" == "native" ]; then + kafka_tarball_filename=(core/build/distributions/kafka*SNAPSHOT.tgz) + if [ ! -e "${kafka_tarball_filename[0]}" ]; then + echo "Kafka tarball not present. Building Kafka tarball for native image." + ./gradlew clean releaseTarGz + fi + + cp core/build/distributions/kafka*SNAPSHOT.tgz "${tmp_native_dir}"/kafka.tgz + cp -r docker/native/native-image-configs "${tmp_native_dir}" + cp docker/native/native_command.sh "${tmp_native_dir}" + fi + must_popd +} + ducker_up() { require_commands docker while [[ $# -ge 1 ]]; do @@ -314,11 +348,13 @@ ducker_up() { -n|--num-nodes) set_once num_nodes "${2}" "number of nodes"; shift 2;; -j|--jdk) set_once jdk_version "${2}" "the OpenJDK base image"; shift 2;; -e|--expose-ports) set_once expose_ports "${2}" "the ports to expose"; shift 2;; + -m|--kafka_mode) set_once kafka_mode "${2}" "the mode in which kafka will run"; shift 2;; *) set_once image_name "${1}" "docker image name"; shift;; esac done [[ -n "${num_nodes}" ]] || num_nodes="${default_num_nodes}" [[ -n "${jdk_version}" ]] || jdk_version="${default_jdk}" + [[ -n "${kafka_mode}" ]] || kafka_mode="${default_kafka_mode}" [[ -n "${image_name}" ]] || image_name="${default_image_name}-${jdk_version/:/-}" [[ "${num_nodes}" =~ ^-?[0-9]+$ ]] || \ die "ducker_up: the number of nodes must be an integer." @@ -334,7 +370,9 @@ use only ${num_nodes}." docker ps >/dev/null || die "ducker_up: failed to run docker. Please check that the daemon is started." + prepare_native_dir ducker_build "${image_name}" + cleanup_native_dir docker inspect --format='{{.Config.Labels}}' --type=image "${image_name}" | grep -q 'ducker.type' local docker_status=${PIPESTATUS[0]} @@ -385,6 +423,10 @@ attempting to start new ones." [[ $? -ne 0 ]] && die "failed to append to the /etc/hosts file on ${node}" done + if [ "$kafka_mode" == "native" ]; then + docker exec --user=root ducker01 bash -c 'cp /opt/kafka-binary/kafka.Kafka /opt/kafka-dev/kafka.Kafka' + fi + echo "ducker_up: added the latest entries to /etc/hosts on each node." generate_cluster_json_file "${num_nodes}" "${ducker_dir}/build/cluster.json" echo "ducker_up: successfully wrote ${ducker_dir}/build/cluster.json" diff --git a/tests/docker/run_tests.sh b/tests/docker/run_tests.sh index 0128fd6897ad9..3285e9ab92007 100755 --- a/tests/docker/run_tests.sh +++ b/tests/docker/run_tests.sh @@ -25,12 +25,21 @@ die() { exit 1 } +if [[ "$_DUCKTAPE_OPTIONS" == *"kafka_mode"* && "$_DUCKTAPE_OPTIONS" == *"native"* ]]; then + export KAFKA_MODE="native" +else + export KAFKA_MODE="jvm" +fi + if [ "$REBUILD" == "t" ]; then ./gradlew clean systemTestLibs + if [ "$KAFKA_MODE" == "native" ]; then + ./gradlew clean releaseTarGz + fi fi if ${SCRIPT_DIR}/ducker-ak ssh | grep -q '(none)'; then - ${SCRIPT_DIR}/ducker-ak up -n "${KAFKA_NUM_CONTAINERS}" || die "ducker-ak up failed" + ${SCRIPT_DIR}/ducker-ak up -n "${KAFKA_NUM_CONTAINERS}" -m "${KAFKA_MODE}" || die "ducker-ak up failed" fi [[ -n ${_DUCKTAPE_OPTIONS} ]] && _DUCKTAPE_OPTIONS="-- ${_DUCKTAPE_OPTIONS}" diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py index 937cdd2fbb91c..a4a73838de006 100644 --- a/tests/kafkatest/__init__.py +++ b/tests/kafkatest/__init__.py @@ -22,4 +22,4 @@ # Instead, in development branches, the version should have a suffix of the form ".devN" # # For example, when Kafka is at version 1.0.0-SNAPSHOT, this should be something like "1.0.0.dev0" -__version__ = '3.8.0.dev0' +__version__ = '3.9.0.dev0' diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py index 335cb02bcc0cb..fe1e3491b62dc 100644 --- a/tests/kafkatest/services/kafka/config_property.py +++ b/tests/kafkatest/services/kafka/config_property.py @@ -70,6 +70,9 @@ SASL_ENABLED_MECHANISMS="sasl.enabled.mechanisms" NEW_GROUP_COORDINATOR_ENABLE="group.coordinator.new.enable" +GROUP_COORDINATOR_REBALANCE_PROTOCOLS="group.coordinator.rebalance.protocols" + +UNSTABLE_FEATURE_VERSIONS_ENABLE="unstable.feature.versions.enable" """ From KafkaConfig.scala diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 7e7f9e104f7e5..7425ac551ca0e 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -409,6 +409,12 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI self.interbroker_sasl_mechanism = interbroker_sasl_mechanism self._security_config = None + # When the new group coordinator is enabled, the new consumer rebalance + # protocol is enabled too. + rebalance_protocols = "classic" + if self.use_new_coordinator: + rebalance_protocols = "classic,consumer" + for node in self.nodes: node_quorum_info = quorum.NodeQuorumInfo(self.quorum_info, node) @@ -422,7 +428,9 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI kraft_broker_configs = { config_property.PORT: config_property.FIRST_BROKER_PORT, config_property.NODE_ID: self.idx(node), - config_property.NEW_GROUP_COORDINATOR_ENABLE: use_new_coordinator + config_property.UNSTABLE_FEATURE_VERSIONS_ENABLE: use_new_coordinator, + config_property.NEW_GROUP_COORDINATOR_ENABLE: use_new_coordinator, + config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS: rebalance_protocols } kraft_broker_plus_zk_configs = kraft_broker_configs.copy() kraft_broker_plus_zk_configs.update(zk_broker_configs) @@ -781,7 +789,9 @@ def prop_file(self, node): override_configs[config_property.ZOOKEEPER_SSL_CLIENT_ENABLE] = 'false' if self.use_new_coordinator: + override_configs[config_property.UNSTABLE_FEATURE_VERSIONS_ENABLE] = 'true' override_configs[config_property.NEW_GROUP_COORDINATOR_ENABLE] = 'true' + override_configs[config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS] = 'classic,consumer' for prop in self.server_prop_overrides: override_configs[prop[0]] = prop[1] @@ -806,7 +816,13 @@ def render_configs(self, configs): return s def start_cmd(self, node): - cmd = "export JMX_PORT=%d; " % self.jmx_port + """ + To bring up kafka using native image, pass following in ducktape options + --globals '{"kafka_mode": "native"}' + """ + kafka_mode = self.context.globals.get("kafka_mode", "") + cmd = f"export KAFKA_MODE={kafka_mode}; " + cmd += "export JMX_PORT=%d; " % self.jmx_port cmd += "export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % self.LOG4J_CONFIG heap_kafka_opts = "-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=%s" % \ self.logs["kafka_heap_dump_file"]["path"] @@ -878,6 +894,10 @@ def start_node(self, node, timeout_sec=60, **kwargs): # format log directories if necessary kafka_storage_script = self.path.script("kafka-storage.sh", node) cmd = "%s format --ignore-formatted --config %s --cluster-id %s" % (kafka_storage_script, KafkaService.CONFIG_FILE, config_property.CLUSTER_ID) + + if self.use_new_coordinator: + cmd += " -f group.version=1" + self.logger.info("Running log directory format command...\n%s" % cmd) node.account.ssh(cmd) @@ -926,7 +946,7 @@ def run_features_command(self, op, new_version): def pids(self, node): """Return process ids associated with running processes on the given node.""" try: - cmd = "jcmd | grep -e %s | awk '{print $1}'" % self.java_class_name() + cmd = "ps ax | grep -i %s | grep -v grep | awk '{print $1}'" % self.java_class_name() pid_arr = [pid for pid in node.account.ssh_capture(cmd, allow_fail=True, callback=int)] return pid_arr except (RemoteCommandError, ValueError) as e: @@ -994,7 +1014,7 @@ def thread_dump(self, node): def clean_node(self, node): JmxMixin.clean_node(self, node) self.security_config.clean_node(node) - node.account.kill_java_processes(self.java_class_name(), + node.account.kill_process(self.java_class_name(), clean_shutdown=False, allow_fail=True) node.account.ssh("sudo rm -rf -- %s" % KafkaService.PERSISTENT_ROOT, allow_fail=False) diff --git a/tests/kafkatest/services/security/templates/jaas.conf b/tests/kafkatest/services/security/templates/jaas.conf index e4fe72cd86088..709dcacd8921c 100644 --- a/tests/kafkatest/services/security/templates/jaas.conf +++ b/tests/kafkatest/services/security/templates/jaas.conf @@ -55,6 +55,7 @@ KafkaServer { useKeyTab=true storeKey=true keyTab="/mnt/security/keytab" + refreshKrb5Config=true principal="kafka/{{ node.account.hostname }}@EXAMPLE.COM"; {% endif %} {% endif %} @@ -81,6 +82,7 @@ Client { principal="zkclient@EXAMPLE.COM"; {% else %} com.sun.security.auth.module.Krb5LoginModule required + refreshKrb5Config=true useKeyTab=true keyTab="/mnt/security/keytab" storeKey=true diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py index 3c6584a3713c6..0a2f852ebbe18 100644 --- a/tests/kafkatest/version.py +++ b/tests/kafkatest/version.py @@ -269,3 +269,7 @@ def get_version(node=None): # 3.7.x version V_3_7_0 = KafkaVersion("3.7.0") LATEST_3_7 = V_3_7_0 + +# 3.8.x version +V_3_8_0 = KafkaVersion("3.8.0") +LATEST_3_8 = V_3_8_0 diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java index aa37919515450..3d8aa93af6d6c 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java @@ -332,7 +332,7 @@ private long parseTimeoutMs() { private MessageFormatter buildFormatter() { MessageFormatter formatter = null; try { - Class messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt)); + Class messageFormatterClass = Class.forName(convertDeprecatedClass(options.valueOf(messageFormatterOpt))); formatter = (MessageFormatter) messageFormatterClass.getDeclaredConstructor().newInstance(); Properties formatterArgs = formatterArgs(); @@ -349,6 +349,25 @@ private MessageFormatter buildFormatter() { return formatter; } + private static String convertDeprecatedClass(String className) { + switch (className) { + case "kafka.tools.DefaultMessageFormatter": + System.err.println("WARNING: kafka.tools.DefaultMessageFormatter is deprecated and will be removed in the next major release. " + + "Please use org.apache.kafka.tools.consumer.DefaultMessageFormatter instead"); + return DefaultMessageFormatter.class.getName(); + case "kafka.tools.LoggingMessageFormatter": + System.err.println("WARNING: kafka.tools.LoggingMessageFormatter is deprecated and will be removed in the next major release. " + + "Please use org.apache.kafka.tools.consumer.LoggingMessageFormatter instead"); + return LoggingMessageFormatter.class.getName(); + case "kafka.tools.NoOpMessageFormatter": + System.err.println("WARNING: kafka.tools.NoOpMessageFormatter is deprecated and will be removed in the next major release. " + + "Please use org.apache.kafka.tools.consumer.NoOpMessageFormatter instead"); + return NoOpMessageFormatter.class.getName(); + default: + return className; + } + } + Properties consumerProps() { return consumerProps; } diff --git a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java index a89782797c1b3..b6d4cea0edbc9 100644 --- a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java @@ -19,7 +19,10 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; + import kafka.test.ClusterInstance; import kafka.test.annotation.ClusterTest; import kafka.test.annotation.Type; @@ -58,9 +61,15 @@ public void testDescribeWithKRaft(ClusterInstance cluster) { String commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(0, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), "describe")) ); + + List features = Arrays.stream(commandOutput.split("\n")).sorted().collect(Collectors.toList()); + + assertEquals("Feature: group.version\tSupportedMinVersion: 0\t" + + "SupportedMaxVersion: 1\tFinalizedVersionLevel: 0\t", outputWithoutEpoch(features.get(0))); + // Change expected message to reflect latest MetadataVersion (SupportedMaxVersion increases when adding a new version) assertEquals("Feature: metadata.version\tSupportedMinVersion: 3.0-IV1\t" + - "SupportedMaxVersion: 3.8-IV0\tFinalizedVersionLevel: 3.3-IV1\t", outputWithoutEpoch(commandOutput)); + "SupportedMaxVersion: 4.0-IV0\tFinalizedVersionLevel: 3.3-IV1\t", outputWithoutEpoch(features.get(1))); } @ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_3_7_IV4) @@ -68,9 +77,15 @@ public void testDescribeWithKRaftAndBootstrapControllers(ClusterInstance cluster String commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(0, FeatureCommand.mainNoExit("--bootstrap-controller", cluster.bootstrapControllers(), "describe")) ); + + List features = Arrays.stream(commandOutput.split("\n")).sorted().collect(Collectors.toList()); + + assertEquals("Feature: group.version\tSupportedMinVersion: 0\t" + + "SupportedMaxVersion: 1\tFinalizedVersionLevel: 0\t", outputWithoutEpoch(features.get(0))); + // Change expected message to reflect latest MetadataVersion (SupportedMaxVersion increases when adding a new version) assertEquals("Feature: metadata.version\tSupportedMinVersion: 3.0-IV1\t" + - "SupportedMaxVersion: 3.8-IV0\tFinalizedVersionLevel: 3.7-IV4\t", outputWithoutEpoch(commandOutput)); + "SupportedMaxVersion: 4.0-IV0\tFinalizedVersionLevel: 3.7-IV4\t", outputWithoutEpoch(features.get(1))); } @ClusterTest(types = {Type.ZK}, metadataVersion = MetadataVersion.IBP_3_3_IV1) @@ -129,7 +144,7 @@ public void testDowngradeMetadataVersionWithKRaft(ClusterInstance cluster) { ); // Change expected message to reflect possible MetadataVersion range 1-N (N increases when adding a new version) assertEquals("Could not disable metadata.version. Invalid update version 0 for feature " + - "metadata.version. Local controller 3000 only supports versions 1-20", commandOutput); + "metadata.version. Local controller 3000 only supports versions 1-21", commandOutput); commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(1, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java index c7d3a82232b76..08029bc1d82ba 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java @@ -645,4 +645,31 @@ public void testParseTimeoutMs() throws Exception { }; assertEquals(100, new ConsoleConsumerOptions(validTimeoutMs).timeoutMs()); } + + @Test + public void testParseDeprecatedFormatter() throws Exception { + String[] deprecatedDefaultMessageFormatter = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--partition", "0", + "--formatter", "kafka.tools.DefaultMessageFormatter", + }; + assertInstanceOf(DefaultMessageFormatter.class, new ConsoleConsumerOptions(deprecatedDefaultMessageFormatter).formatter()); + + String[] deprecatedLoggingMessageFormatter = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--partition", "0", + "--formatter", "kafka.tools.LoggingMessageFormatter", + }; + assertInstanceOf(LoggingMessageFormatter.class, new ConsoleConsumerOptions(deprecatedLoggingMessageFormatter).formatter()); + + String[] deprecatedNoOpMessageFormatter = new String[]{ + "--bootstrap-server", "localhost:9092", + "--topic", "test", + "--partition", "0", + "--formatter", "kafka.tools.NoOpMessageFormatter", + }; + assertInstanceOf(NoOpMessageFormatter.class, new ConsoleConsumerOptions(deprecatedNoOpMessageFormatter).formatter()); + } } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java index ae46b7ac2ff40..4fbdeec0d946f 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTest.java @@ -38,7 +38,6 @@ import java.time.Duration; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -86,7 +85,10 @@ public Seq generateConfigs() { 0, false ).foreach(props -> { - props.setProperty(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG, isNewGroupCoordinatorEnabled() + ""); + if (isNewGroupCoordinatorEnabled()) { + props.setProperty(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true"); + props.setProperty(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer"); + } cfgs.add(KafkaConfig.fromProps(props)); return null; }); @@ -139,10 +141,6 @@ ConsumerGroupCommand.ConsumerGroupService getConsumerGroupService(String[] args) return service; } - ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers) { - return addConsumerGroupExecutor(numConsumers, TOPIC, GROUP, RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), false, GroupProtocol.CLASSIC.name); - } - ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers, String groupProtocol) { return addConsumerGroupExecutor(numConsumers, TOPIC, GROUP, RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), false, groupProtocol); } @@ -151,10 +149,6 @@ ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers, String groupPro return addConsumerGroupExecutor(numConsumers, TOPIC, GROUP, RangeAssignor.class.getName(), remoteAssignor, Optional.empty(), false, groupProtocol); } - ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers, String group, String groupProtocol) { - return addConsumerGroupExecutor(numConsumers, TOPIC, group, RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), false, groupProtocol); - } - ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers, String topic, String group, String groupProtocol) { return addConsumerGroupExecutor(numConsumers, topic, group, RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), false, groupProtocol); } @@ -167,10 +161,6 @@ ConsumerGroupExecutor addConsumerGroupExecutor(int numConsumers, String topic, S return executor; } - SimpleConsumerGroupExecutor addSimpleGroupExecutor(String group) { - return addSimpleGroupExecutor(Arrays.asList(new TopicPartition(TOPIC, 0)), group); - } - SimpleConsumerGroupExecutor addSimpleGroupExecutor(Collection partitions, String group) { SimpleConsumerGroupExecutor executor = new SimpleConsumerGroupExecutor(bootstrapServers(listenerName()), group, partitions); addExecutor(executor); @@ -339,14 +329,6 @@ public static Stream getTestQuorumAndGroupProtocolParametersAll() { return BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll(); } - public static Stream getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly() { - return BaseConsumerTest.getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly(); - } - - public static Stream getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly() { - return BaseConsumerTest.getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly(); - } - @SuppressWarnings({"deprecation"}) static Seq seq(Collection seq) { return JavaConverters.asScalaIteratorConverter(seq.iterator()).asScala().toSeq(); diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java index ffd33789c9c18..e5177ddaead2c 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java @@ -19,64 +19,118 @@ import kafka.test.ClusterConfig; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.common.Features; +import org.apache.kafka.server.common.GroupVersion; import java.time.Duration; -import java.util.Map; -import java.util.HashMap; import java.util.ArrayList; -import java.util.List; import java.util.Collections; -import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; -import static java.util.Collections.singleton; import static kafka.test.annotation.Type.CO_KRAFT; import static kafka.test.annotation.Type.KRAFT; import static kafka.test.annotation.Type.ZK; +import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG; import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG; import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG; import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG; +/** + * The old test framework {@link kafka.api.BaseConsumerTest#getTestQuorumAndGroupProtocolParametersAll} test for the following cases: + *

        + *
      • (ZK / KRAFT servers) with (group.coordinator.new.enable=false) with (classic group protocol) = 2 cases
      • + *
      • (KRAFT server) with (group.coordinator.new.enable=true) with (classic group protocol) = 1 case
      • + *
      • (KRAFT server) with (group.coordinator.new.enable=true) with (consumer group protocol) = 1 case
      • + *
      + *

      + * The new test framework run seven cases for the following cases: + *

        + *
      • (ZK / KRAFT / CO_KRAFT servers) with (group.coordinator.new.enable=false) with (classic group protocol) = 3 cases
      • + *
      • (KRAFT / CO_KRAFT servers) with (group.coordinator.new.enable=true) with (classic group protocol) = 2 cases
      • + *
      • (KRAFT / CO_KRAFT servers) with (group.coordinator.new.enable=true) with (consumer group protocol) = 2 cases
      • + *
      + *

      + * We can reduce the number of cases as same as the old test framework by using the following methods: + *

        + *
      • {@link #forConsumerGroupCoordinator} for the case of (consumer group protocol)
      • + *
      • (CO_KRAFT servers) with (group.coordinator.new.enable=true) with (classic / consumer group protocols) = 2 cases
      • + *
      + *
        + *
      • {@link #forClassicGroupCoordinator} for the case of (classic group protocol)
      • + *
      • (ZK / KRAFT servers) with (group.coordinator.new.enable=false) with (classic group protocol) = 2 cases
      • + *
      + */ class ConsumerGroupCommandTestUtils { private ConsumerGroupCommandTestUtils() { } static List generator() { + return Stream.concat(forConsumerGroupCoordinator().stream(), forClassicGroupCoordinator().stream()) + .collect(Collectors.toList()); + } + + static List forConsumerGroupCoordinator() { Map serverProperties = new HashMap<>(); serverProperties.put(OFFSETS_TOPIC_PARTITIONS_CONFIG, "1"); serverProperties.put(OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1"); - serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "false"); + serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true"); + serverProperties.put(GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer"); - ClusterConfig classicGroupCoordinator = ClusterConfig.defaultBuilder() - .setTypes(Stream.of(ZK, KRAFT, CO_KRAFT).collect(Collectors.toSet())) + return Collections.singletonList(ClusterConfig.defaultBuilder() + .setFeatures(Collections.singletonMap(Features.GROUP_VERSION, GroupVersion.GV_1.featureLevel())) + .setTypes(Collections.singleton(CO_KRAFT)) .setServerProperties(serverProperties) - .setTags(Collections.singletonList("classicGroupCoordinator")) - .build(); + .setTags(Collections.singletonList("consumerGroupCoordinator")) + .build()); + } - // Following are test case config with new group coordinator - serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true"); + static List forClassicGroupCoordinator() { + Map serverProperties = new HashMap<>(); + serverProperties.put(OFFSETS_TOPIC_PARTITIONS_CONFIG, "1"); + serverProperties.put(OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1"); + serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "false"); - ClusterConfig consumerGroupCoordinator = ClusterConfig.defaultBuilder() - .setTypes(Stream.of(KRAFT, CO_KRAFT).collect(Collectors.toSet())) + return Collections.singletonList(ClusterConfig.defaultBuilder() + .setTypes(Stream.of(ZK, KRAFT).collect(Collectors.toSet())) .setServerProperties(serverProperties) - .setTags(Collections.singletonList("newGroupCoordinator")) - .build(); - return Arrays.asList(classicGroupCoordinator, consumerGroupCoordinator); + .setTags(Collections.singletonList("classicGroupCoordinator")) + .build()); + } + + static AutoCloseable buildConsumers(int numberOfConsumers, + Set partitions, + Supplier> consumerSupplier) { + return buildConsumers(numberOfConsumers, false, consumerSupplier, + consumer -> consumer.assign(partitions)); } static AutoCloseable buildConsumers(int numberOfConsumers, boolean syncCommit, String topic, Supplier> consumerSupplier) { + return buildConsumers(numberOfConsumers, syncCommit, consumerSupplier, + consumer -> consumer.subscribe(Collections.singleton(topic))); + } + + private static AutoCloseable buildConsumers(int numberOfConsumers, + boolean syncCommit, + Supplier> consumerSupplier, + Consumer> setPartitions) { List> consumers = new ArrayList<>(numberOfConsumers); ExecutorService executor = Executors.newFixedThreadPool(numberOfConsumers); AtomicBoolean closed = new AtomicBoolean(false); @@ -85,7 +139,10 @@ static AutoCloseable buildConsumers(int numberOfConsumers, for (int i = 0; i < numberOfConsumers; i++) { KafkaConsumer consumer = consumerSupplier.get(); consumers.add(consumer); - executor.execute(() -> initConsumer(topic, syncCommit, consumer, closed)); + executor.execute(() -> initConsumer(syncCommit, () -> { + setPartitions.accept(consumer); + return consumer; + }, closed)); } return closeable; } catch (Throwable e) { @@ -101,13 +158,14 @@ private static void releaseConsumers(AtomicBoolean closed, List void initConsumer(String topic, boolean syncCommit, KafkaConsumer consumer, AtomicBoolean closed) { - try (KafkaConsumer kafkaConsumer = consumer) { - kafkaConsumer.subscribe(singleton(topic)); + private static void initConsumer(boolean syncCommit, + Supplier> consumerSupplier, + AtomicBoolean closed) { + try (KafkaConsumer kafkaConsumer = consumerSupplier.get()) { while (!closed.get()) { - consumer.poll(Duration.ofMillis(Long.MAX_VALUE)); + kafkaConsumer.poll(Duration.ofMillis(Long.MAX_VALUE)); if (syncCommit) - consumer.commitSync(); + kafkaConsumer.commitSync(); } } catch (WakeupException e) { // OK diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java index 6fd2928da9f2f..068718a9fc69a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java @@ -17,449 +17,537 @@ package org.apache.kafka.tools.consumer.group; import joptsimple.OptionException; +import kafka.test.ClusterConfig; +import kafka.test.ClusterInstance; +import kafka.test.annotation.ClusterTemplate; +import kafka.test.junit.ClusterTestExtensions; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.ConsumerGroupListing; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.GroupProtocol; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.RangeAssignor; import org.apache.kafka.common.ConsumerGroupState; import org.apache.kafka.common.GroupType; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.StringDeserializer; +import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; + import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.apache.kafka.tools.ToolsTestUtils; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.api.extension.ExtendWith; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.EnumSet; +import java.util.HashMap; import java.util.HashSet; +import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.Properties; import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; -import static org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES; +import static java.util.Collections.emptyMap; import static org.apache.kafka.common.utils.Utils.mkSet; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; -public class ListConsumerGroupTest extends ConsumerGroupCommandTest { - @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) - @MethodSource("getTestQuorumAndGroupProtocolParametersAll") - public void testListConsumerGroupsWithoutFilters(String quorum, String groupProtocol) throws Exception { - String simpleGroup = "simple-group"; +@Tag("integration") +@ExtendWith(ClusterTestExtensions.class) +public class ListConsumerGroupTest { + private final static String TOPIC_PREFIX = "test.topic."; + private final static String TOPIC_PARTITIONS_GROUP_PREFIX = "test.topic.partitions.group."; + private final static String TOPIC_GROUP_PREFIX = "test.topic.group."; + private final static String PROTOCOL_GROUP_PREFIX = "test.protocol.group."; + private final ClusterInstance clusterInstance; + + ListConsumerGroupTest(ClusterInstance clusterInstance) { + this.clusterInstance = clusterInstance; + } - createOffsetsTopic(listenerName(), new Properties()); + private static List defaultGenerator() { + return ConsumerGroupCommandTestUtils.generator(); + } - addSimpleGroupExecutor(simpleGroup); - addConsumerGroupExecutor(1); - addConsumerGroupExecutor(1, PROTOCOL_GROUP, groupProtocol); + private static List consumerProtocolOnlyGenerator() { + return ConsumerGroupCommandTestUtils.forConsumerGroupCoordinator(); + } - String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list"}; - ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); + private List supportedGroupProtocols() { + return new ArrayList<>(clusterInstance.supportedGroupProtocols()); + } - Set expectedGroups = set(Arrays.asList(GROUP, simpleGroup, PROTOCOL_GROUP)); - final AtomicReference foundGroups = new AtomicReference<>(); + @ClusterTemplate("defaultGenerator") + public void testListConsumerGroupsWithoutFilters() throws Exception { + for (int i = 0; i < supportedGroupProtocols().size(); i++) { + GroupProtocol groupProtocol = supportedGroupProtocols().get(i); + String topic = TOPIC_PREFIX + groupProtocol.name; + String protocolGroup = PROTOCOL_GROUP_PREFIX + groupProtocol.name; + String topicGroup = TOPIC_GROUP_PREFIX + i; + String topicPartitionsGroup = TOPIC_PARTITIONS_GROUP_PREFIX + i; + createTopic(topic); + + try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Collections.singleton(new TopicPartition(topic, 0))); + AutoCloseable topicConsumerGroupExecutor = consumerGroupClosable(GroupProtocol.CLASSIC, topicGroup, topic); + AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, protocolGroup, topic); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--list"}); + ) { + Set expectedGroups = set(Arrays.asList(topicPartitionsGroup, topicGroup, protocolGroup)); + final AtomicReference foundGroups = new AtomicReference<>(); + + TestUtils.waitForCondition(() -> { + foundGroups.set(set(service.listConsumerGroups())); + return Objects.equals(expectedGroups, foundGroups.get()); + }, "Expected --list to show groups " + expectedGroups + ", but found " + foundGroups.get() + "."); + } - TestUtils.waitForCondition(() -> { - foundGroups.set(set(service.listConsumerGroups())); - return Objects.equals(expectedGroups, foundGroups.get()); - }, "Expected --list to show groups " + expectedGroups + ", but found " + foundGroups.get() + "."); + removeConsumer(set(Arrays.asList(topicPartitionsGroup, topicGroup, protocolGroup))); + deleteTopic(topic); + } } - @Test + @ClusterTemplate("defaultGenerator") public void testListWithUnrecognizedNewConsumerOption() { - String[] cgcArgs = new String[]{"--new-consumer", "--bootstrap-server", bootstrapServers(listenerName()), "--list"}; - assertThrows(OptionException.class, () -> getConsumerGroupService(cgcArgs)); + String[] cgcArgs = new String[]{"--new-consumer", "--bootstrap-server", clusterInstance.bootstrapServers(), "--list"}; + Assertions.assertThrows(OptionException.class, () -> getConsumerGroupService(cgcArgs)); } - @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) - @MethodSource("getTestQuorumAndGroupProtocolParametersAll") - public void testListConsumerGroupsWithStates(String quorum, String groupProtocol) throws Exception { - String simpleGroup = "simple-group"; - - createOffsetsTopic(listenerName(), new Properties()); - - addSimpleGroupExecutor(simpleGroup); - addConsumerGroupExecutor(1, groupProtocol); - - String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state"}; - ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); - - Set expectedListing = mkSet( - new ConsumerGroupListing( - simpleGroup, - true, - Optional.of(ConsumerGroupState.EMPTY), - Optional.of(GroupType.CLASSIC) - ), - new ConsumerGroupListing( - GROUP, - false, - Optional.of(ConsumerGroupState.STABLE), - Optional.of(GroupType.parse(groupProtocol)) - ) - ); - - assertGroupListing( - service, - Collections.emptySet(), - EnumSet.allOf(ConsumerGroupState.class), - expectedListing - ); - - expectedListing = mkSet( - new ConsumerGroupListing( - GROUP, - false, - Optional.of(ConsumerGroupState.STABLE), - Optional.of(GroupType.parse(groupProtocol)) - ) - ); - - assertGroupListing( - service, - Collections.emptySet(), - mkSet(ConsumerGroupState.STABLE), - expectedListing - ); + @ClusterTemplate("defaultGenerator") + public void testListConsumerGroupsWithStates() throws Exception { + for (int i = 0; i < supportedGroupProtocols().size(); i++) { + GroupProtocol groupProtocol = supportedGroupProtocols().get(i); + String topic = TOPIC_PREFIX + groupProtocol.name; + String protocolGroup = PROTOCOL_GROUP_PREFIX + groupProtocol.name; + String topicPartitionsGroup = TOPIC_PARTITIONS_GROUP_PREFIX + i; + createTopic(topic); + + try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Collections.singleton(new TopicPartition(topic, 0))); + AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, protocolGroup, topic); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state"}); + ) { + Set expectedListing = mkSet( + new ConsumerGroupListing( + topicPartitionsGroup, + true, + Optional.of(ConsumerGroupState.EMPTY), + Optional.of(GroupType.CLASSIC) + ), + new ConsumerGroupListing( + protocolGroup, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.parse(groupProtocol.name())) + ) + ); + + assertGroupListing( + service, + Collections.emptySet(), + EnumSet.allOf(ConsumerGroupState.class), + expectedListing + ); + + expectedListing = mkSet( + new ConsumerGroupListing( + protocolGroup, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.parse(groupProtocol.name())) + ) + ); + + assertGroupListing( + service, + Collections.emptySet(), + mkSet(ConsumerGroupState.STABLE), + expectedListing + ); + + assertGroupListing( + service, + Collections.emptySet(), + mkSet(ConsumerGroupState.PREPARING_REBALANCE), + Collections.emptySet() + ); + } - assertGroupListing( - service, - Collections.emptySet(), - mkSet(ConsumerGroupState.PREPARING_REBALANCE), - Collections.emptySet() - ); + removeConsumer(set(Arrays.asList(topicPartitionsGroup, protocolGroup))); + deleteTopic(topic); + } } - @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) - @MethodSource("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly") - public void testListConsumerGroupsWithTypesClassicProtocol(String quorum, String groupProtocol) throws Exception { - String simpleGroup = "simple-group"; - - createOffsetsTopic(listenerName(), new Properties()); - - addSimpleGroupExecutor(simpleGroup); - addConsumerGroupExecutor(1); - - String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list"}; - ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); - - Set expectedListing = mkSet( - new ConsumerGroupListing( - simpleGroup, - true, - Optional.of(ConsumerGroupState.EMPTY), - Optional.of(GroupType.CLASSIC) - ), - new ConsumerGroupListing( - GROUP, - false, - Optional.of(ConsumerGroupState.STABLE), - Optional.of(GroupType.CLASSIC) - ) - ); - - // No filters explicitly mentioned. Expectation is that all groups are returned. - assertGroupListing( - service, - Collections.emptySet(), - Collections.emptySet(), - expectedListing - ); - - // When group type is mentioned: - // Old Group Coordinator returns empty listings if the type is not Classic. - // New Group Coordinator returns groups according to the filter. - assertGroupListing( - service, - mkSet(GroupType.CONSUMER), - Collections.emptySet(), - Collections.emptySet() - ); - - assertGroupListing( - service, - mkSet(GroupType.CLASSIC), - Collections.emptySet(), - expectedListing - ); + @ClusterTemplate("defaultGenerator") + public void testListConsumerGroupsWithTypesClassicProtocol() throws Exception { + GroupProtocol groupProtocol = GroupProtocol.CLASSIC; + String topic = TOPIC_PREFIX + groupProtocol.name; + String protocolGroup = PROTOCOL_GROUP_PREFIX + groupProtocol.name; + String topicPartitionsGroup = TOPIC_PARTITIONS_GROUP_PREFIX + "0"; + createTopic(topic); + + try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Collections.singleton(new TopicPartition(topic, 0))); + AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, protocolGroup, topic); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state"}); + ) { + Set expectedListing = mkSet( + new ConsumerGroupListing( + topicPartitionsGroup, + true, + Optional.of(ConsumerGroupState.EMPTY), + Optional.of(GroupType.CLASSIC) + ), + new ConsumerGroupListing( + protocolGroup, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.CLASSIC) + ) + ); + + // No filters explicitly mentioned. Expectation is that all groups are returned. + assertGroupListing( + service, + Collections.emptySet(), + Collections.emptySet(), + expectedListing + ); + + // When group type is mentioned: + // Old Group Coordinator returns empty listings if the type is not Classic. + // New Group Coordinator returns groups according to the filter. + assertGroupListing( + service, + mkSet(GroupType.CONSUMER), + Collections.emptySet(), + Collections.emptySet() + ); + + assertGroupListing( + service, + mkSet(GroupType.CLASSIC), + Collections.emptySet(), + expectedListing + ); + } } - @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) - @MethodSource("getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly") - public void testListConsumerGroupsWithTypesConsumerProtocol(String quorum, String groupProtocol) throws Exception { - String simpleGroup = "simple-group"; - - createOffsetsTopic(listenerName(), new Properties()); - - addSimpleGroupExecutor(simpleGroup); - addConsumerGroupExecutor(1); - addConsumerGroupExecutor(1, PROTOCOL_GROUP, groupProtocol); - - String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list"}; - ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs); - - // No filters explicitly mentioned. Expectation is that all groups are returned. - Set expectedListing = mkSet( - new ConsumerGroupListing( - simpleGroup, - true, - Optional.of(ConsumerGroupState.EMPTY), - Optional.of(GroupType.CLASSIC) - ), - new ConsumerGroupListing( - GROUP, - false, - Optional.of(ConsumerGroupState.STABLE), - Optional.of(GroupType.CLASSIC) - ), - new ConsumerGroupListing( - PROTOCOL_GROUP, - false, - Optional.of(ConsumerGroupState.STABLE), - Optional.of(GroupType.CONSUMER) - ) - ); - - assertGroupListing( - service, - Collections.emptySet(), - Collections.emptySet(), - expectedListing - ); - - // When group type is mentioned: - // New Group Coordinator returns groups according to the filter. - expectedListing = mkSet( - new ConsumerGroupListing( - PROTOCOL_GROUP, - false, - Optional.of(ConsumerGroupState.STABLE), - Optional.of(GroupType.CONSUMER) - ) - ); - - assertGroupListing( - service, - mkSet(GroupType.CONSUMER), - Collections.emptySet(), - expectedListing - ); - - expectedListing = mkSet( - new ConsumerGroupListing( - simpleGroup, - true, - Optional.of(ConsumerGroupState.EMPTY), - Optional.of(GroupType.CLASSIC) - ), - new ConsumerGroupListing( - GROUP, - false, - Optional.of(ConsumerGroupState.STABLE), - Optional.of(GroupType.CLASSIC) - ) - ); - - assertGroupListing( - service, - mkSet(GroupType.CLASSIC), - Collections.emptySet(), - expectedListing - ); + @ClusterTemplate("consumerProtocolOnlyGenerator") + public void testListConsumerGroupsWithTypesConsumerProtocol() throws Exception { + GroupProtocol groupProtocol = GroupProtocol.CONSUMER; + String topic = TOPIC_PREFIX + groupProtocol.name; + String protocolGroup = PROTOCOL_GROUP_PREFIX + groupProtocol.name; + String topicGroup = TOPIC_GROUP_PREFIX + "0"; + String topicPartitionsGroup = TOPIC_PARTITIONS_GROUP_PREFIX + "0"; + createTopic(topic); + + try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Collections.singleton(new TopicPartition(topic, 0))); + AutoCloseable topicConsumerGroupExecutor = consumerGroupClosable(GroupProtocol.CLASSIC, topicGroup, topic); + AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, protocolGroup, topic); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--list"}); + ) { + + + // No filters explicitly mentioned. Expectation is that all groups are returned. + Set expectedListing = mkSet( + new ConsumerGroupListing( + topicPartitionsGroup, + true, + Optional.of(ConsumerGroupState.EMPTY), + Optional.of(GroupType.CLASSIC) + ), + new ConsumerGroupListing( + topicGroup, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.CLASSIC) + ), + new ConsumerGroupListing( + protocolGroup, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.CONSUMER) + ) + ); + + assertGroupListing( + service, + Collections.emptySet(), + Collections.emptySet(), + expectedListing + ); + + // When group type is mentioned: + // New Group Coordinator returns groups according to the filter. + expectedListing = mkSet( + new ConsumerGroupListing( + protocolGroup, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.CONSUMER) + ) + ); + + assertGroupListing( + service, + mkSet(GroupType.CONSUMER), + Collections.emptySet(), + expectedListing + ); + + expectedListing = mkSet( + new ConsumerGroupListing( + topicPartitionsGroup, + true, + Optional.of(ConsumerGroupState.EMPTY), + Optional.of(GroupType.CLASSIC) + ), + new ConsumerGroupListing( + topicGroup, + false, + Optional.of(ConsumerGroupState.STABLE), + Optional.of(GroupType.CLASSIC) + ) + ); + + assertGroupListing( + service, + mkSet(GroupType.CLASSIC), + Collections.emptySet(), + expectedListing + ); + } } - @Test - public void testConsumerGroupStatesFromString() { - Set result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable"); - assertEquals(set(Collections.singleton(ConsumerGroupState.STABLE)), result); - - result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable, PreparingRebalance"); - assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE, ConsumerGroupState.PREPARING_REBALANCE)), result); - - result = ConsumerGroupCommand.consumerGroupStatesFromString("Dead,CompletingRebalance,"); - assertEquals(set(Arrays.asList(ConsumerGroupState.DEAD, ConsumerGroupState.COMPLETING_REBALANCE)), result); - - result = ConsumerGroupCommand.consumerGroupStatesFromString("stable"); - assertEquals(set(Collections.singletonList(ConsumerGroupState.STABLE)), result); - - result = ConsumerGroupCommand.consumerGroupStatesFromString("stable, assigning"); - assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE, ConsumerGroupState.ASSIGNING)), result); - - result = ConsumerGroupCommand.consumerGroupStatesFromString("dead,reconciling,"); - assertEquals(set(Arrays.asList(ConsumerGroupState.DEAD, ConsumerGroupState.RECONCILING)), result); - - assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupStatesFromString("bad, wrong")); - - assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupStatesFromString(" bad, Stable")); - - assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupStatesFromString(" , ,")); + @ClusterTemplate("defaultGenerator") + public void testListGroupCommandClassicProtocol() throws Exception { + GroupProtocol groupProtocol = GroupProtocol.CLASSIC; + String topic = TOPIC_PREFIX + groupProtocol.name; + String protocolGroup = PROTOCOL_GROUP_PREFIX + groupProtocol.name; + String topicPartitionsGroup = TOPIC_PARTITIONS_GROUP_PREFIX + "0"; + createTopic(topic); + + try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Collections.singleton(new TopicPartition(topic, 0))); + AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, protocolGroup, topic) + ) { + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list"), + Collections.emptyList(), + mkSet( + Collections.singletonList(protocolGroup), + Collections.singletonList(topicPartitionsGroup) + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state"), + Arrays.asList("GROUP", "STATE"), + mkSet( + Arrays.asList(protocolGroup, "Stable"), + Arrays.asList(topicPartitionsGroup, "Empty") + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type"), + Arrays.asList("GROUP", "TYPE"), + mkSet( + Arrays.asList(protocolGroup, "Classic"), + Arrays.asList(topicPartitionsGroup, "Classic") + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "--state"), + Arrays.asList("GROUP", "TYPE", "STATE"), + mkSet( + Arrays.asList(protocolGroup, "Classic", "Stable"), + Arrays.asList(topicPartitionsGroup, "Classic", "Empty") + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state", "Stable"), + Arrays.asList("GROUP", "STATE"), + mkSet( + Arrays.asList(protocolGroup, "Stable") + ) + ); + + // Check case-insensitivity in state filter. + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state", "stable"), + Arrays.asList("GROUP", "STATE"), + mkSet( + Arrays.asList(protocolGroup, "Stable") + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "Classic"), + Arrays.asList("GROUP", "TYPE"), + mkSet( + Arrays.asList(protocolGroup, "Classic"), + Arrays.asList(topicPartitionsGroup, "Classic") + ) + ); + + // Check case-insensitivity in type filter. + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "classic"), + Arrays.asList("GROUP", "TYPE"), + mkSet( + Arrays.asList(protocolGroup, "Classic"), + Arrays.asList(topicPartitionsGroup, "Classic") + ) + ); + } } - @Test - public void testConsumerGroupTypesFromString() { - Set result = ConsumerGroupCommand.consumerGroupTypesFromString("consumer"); - assertEquals(set(Collections.singleton(GroupType.CONSUMER)), result); - - result = ConsumerGroupCommand.consumerGroupTypesFromString("consumer, classic"); - assertEquals(set(Arrays.asList(GroupType.CONSUMER, GroupType.CLASSIC)), result); - - result = ConsumerGroupCommand.consumerGroupTypesFromString("Consumer, Classic"); - assertEquals(set(Arrays.asList(GroupType.CONSUMER, GroupType.CLASSIC)), result); - - assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString("bad, wrong")); - - assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString(" bad, generic")); - - assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString(" , ,")); + @ClusterTemplate("consumerProtocolOnlyGenerator") + public void testListGroupCommandConsumerProtocol() throws Exception { + GroupProtocol groupProtocol = GroupProtocol.CONSUMER; + String topic = TOPIC_PREFIX + groupProtocol.name; + String protocolGroup = PROTOCOL_GROUP_PREFIX + groupProtocol.name; + String topicPartitionsGroup = TOPIC_PARTITIONS_GROUP_PREFIX + "0"; + createTopic(topic); + + try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Collections.singleton(new TopicPartition(topic, 0))); + AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, protocolGroup, topic) + ) { + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list"), + Collections.emptyList(), + mkSet( + Collections.singletonList(protocolGroup), + Collections.singletonList(topicPartitionsGroup) + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state"), + Arrays.asList("GROUP", "STATE"), + mkSet( + Arrays.asList(protocolGroup, "Stable"), + Arrays.asList(topicPartitionsGroup, "Empty") + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type"), + Arrays.asList("GROUP", "TYPE"), + mkSet( + Arrays.asList(protocolGroup, "Consumer"), + Arrays.asList(topicPartitionsGroup, "Classic") + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "--state"), + Arrays.asList("GROUP", "TYPE", "STATE"), + mkSet( + Arrays.asList(protocolGroup, "Consumer", "Stable"), + Arrays.asList(topicPartitionsGroup, "Classic", "Empty") + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "consumer"), + Arrays.asList("GROUP", "TYPE"), + mkSet( + Arrays.asList(protocolGroup, "Consumer") + ) + ); + + validateListOutput( + Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "consumer", "--state", "Stable"), + Arrays.asList("GROUP", "TYPE", "STATE"), + mkSet( + Arrays.asList(protocolGroup, "Consumer", "Stable") + ) + ); + } } - @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) - @MethodSource("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly") - public void testListGroupCommandClassicProtocol(String quorum, String groupProtocol) throws Exception { - String simpleGroup = "simple-group"; - - createOffsetsTopic(listenerName(), new Properties()); - - addSimpleGroupExecutor(simpleGroup); - addConsumerGroupExecutor(1); - - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list"), - Collections.emptyList(), - mkSet( - Collections.singletonList(GROUP), - Collections.singletonList(simpleGroup) - ) - ); - - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state"), - Arrays.asList("GROUP", "STATE"), - mkSet( - Arrays.asList(GROUP, "Stable"), - Arrays.asList(simpleGroup, "Empty") - ) - ); - - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type"), - Arrays.asList("GROUP", "TYPE"), - mkSet( - Arrays.asList(GROUP, "Classic"), - Arrays.asList(simpleGroup, "Classic") - ) + private AutoCloseable consumerGroupClosable(GroupProtocol protocol, String groupId, String topicName) { + Map configs = composeConfigs( + groupId, + protocol.name, + emptyMap() ); - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "--state"), - Arrays.asList("GROUP", "TYPE", "STATE"), - mkSet( - Arrays.asList(GROUP, "Classic", "Stable"), - Arrays.asList(simpleGroup, "Classic", "Empty") - ) - ); - - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state", "Stable"), - Arrays.asList("GROUP", "STATE"), - mkSet( - Arrays.asList(GROUP, "Stable") - ) - ); - - // Check case-insensitivity in state filter. - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state", "stable"), - Arrays.asList("GROUP", "STATE"), - mkSet( - Arrays.asList(GROUP, "Stable") - ) + return ConsumerGroupCommandTestUtils.buildConsumers( + 1, + false, + topicName, + () -> new KafkaConsumer(configs) ); + } - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "Classic"), - Arrays.asList("GROUP", "TYPE"), - mkSet( - Arrays.asList(GROUP, "Classic"), - Arrays.asList(simpleGroup, "Classic") - ) + private AutoCloseable consumerGroupClosable(String groupId, Set topicPartitions) { + Map configs = composeConfigs( + groupId, + GroupProtocol.CLASSIC.name, + emptyMap() ); - // Check case-insensitivity in type filter. - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "classic"), - Arrays.asList("GROUP", "TYPE"), - mkSet( - Arrays.asList(GROUP, "Classic"), - Arrays.asList(simpleGroup, "Classic") - ) + return ConsumerGroupCommandTestUtils.buildConsumers( + 1, + topicPartitions, + () -> new KafkaConsumer(configs) ); } - @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES) - @MethodSource("getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly") - public void testListGroupCommandConsumerProtocol(String quorum, String groupProtocol) throws Exception { - String simpleGroup = "simple-group"; - - createOffsetsTopic(listenerName(), new Properties()); - - addSimpleGroupExecutor(simpleGroup); - addConsumerGroupExecutor(1, PROTOCOL_GROUP, groupProtocol); - - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list"), - Collections.emptyList(), - mkSet( - Collections.singletonList(PROTOCOL_GROUP), - Collections.singletonList(simpleGroup) - ) - ); + private Map composeConfigs(String groupId, String groupProtocol, Map customConfigs) { + Map configs = new HashMap<>(); + configs.put(BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()); + configs.put(GROUP_ID_CONFIG, groupId); + configs.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + configs.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + configs.put(GROUP_PROTOCOL_CONFIG, groupProtocol); + configs.put(PARTITION_ASSIGNMENT_STRATEGY_CONFIG, RangeAssignor.class.getName()); + + configs.putAll(customConfigs); + return configs; + } - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state"), - Arrays.asList("GROUP", "STATE"), - mkSet( - Arrays.asList(PROTOCOL_GROUP, "Stable"), - Arrays.asList(simpleGroup, "Empty") - ) + private ConsumerGroupCommand.ConsumerGroupService getConsumerGroupService(String[] args) { + ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(args); + ConsumerGroupCommand.ConsumerGroupService service = new ConsumerGroupCommand.ConsumerGroupService( + opts, + Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE)) ); - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type"), - Arrays.asList("GROUP", "TYPE"), - mkSet( - Arrays.asList(PROTOCOL_GROUP, "Consumer"), - Arrays.asList(simpleGroup, "Classic") - ) - ); + return service; + } - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "--state"), - Arrays.asList("GROUP", "TYPE", "STATE"), - mkSet( - Arrays.asList(PROTOCOL_GROUP, "Consumer", "Stable"), - Arrays.asList(simpleGroup, "Classic", "Empty") - ) - ); + private void createTopic(String topic) { + try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + Assertions.assertDoesNotThrow(() -> admin.createTopics(Collections.singletonList(new NewTopic(topic, 1, (short) 1))).topicId(topic).get()); + } + } - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "consumer"), - Arrays.asList("GROUP", "TYPE"), - mkSet( - Arrays.asList(PROTOCOL_GROUP, "Consumer") - ) - ); + private void deleteTopic(String topic) { + try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + Assertions.assertDoesNotThrow(() -> admin.deleteTopics(Collections.singleton(topic)).all().get()); + } + } - validateListOutput( - Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--type", "consumer", "--state", "Stable"), - Arrays.asList("GROUP", "TYPE", "STATE"), - mkSet( - Arrays.asList(PROTOCOL_GROUP, "Consumer", "Stable") - ) - ); + private void removeConsumer(Set groupIds) { + try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + Assertions.assertDoesNotThrow(() -> admin.deleteConsumerGroups(groupIds).all().get()); + } } /** @@ -527,3 +615,50 @@ public static Set set(Collection set) { return new HashSet<>(set); } } + +class ListConsumerGroupUnitTest { + @Test + public void testConsumerGroupStatesFromString() { + Set result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable"); + Assertions.assertEquals(ListConsumerGroupTest.set(Collections.singleton(ConsumerGroupState.STABLE)), result); + + result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable, PreparingRebalance"); + Assertions.assertEquals(ListConsumerGroupTest.set(Arrays.asList(ConsumerGroupState.STABLE, ConsumerGroupState.PREPARING_REBALANCE)), result); + + result = ConsumerGroupCommand.consumerGroupStatesFromString("Dead,CompletingRebalance,"); + Assertions.assertEquals(ListConsumerGroupTest.set(Arrays.asList(ConsumerGroupState.DEAD, ConsumerGroupState.COMPLETING_REBALANCE)), result); + + result = ConsumerGroupCommand.consumerGroupStatesFromString("stable"); + Assertions.assertEquals(ListConsumerGroupTest.set(Collections.singletonList(ConsumerGroupState.STABLE)), result); + + result = ConsumerGroupCommand.consumerGroupStatesFromString("stable, assigning"); + Assertions.assertEquals(ListConsumerGroupTest.set(Arrays.asList(ConsumerGroupState.STABLE, ConsumerGroupState.ASSIGNING)), result); + + result = ConsumerGroupCommand.consumerGroupStatesFromString("dead,reconciling,"); + Assertions.assertEquals(ListConsumerGroupTest.set(Arrays.asList(ConsumerGroupState.DEAD, ConsumerGroupState.RECONCILING)), result); + + Assertions.assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupStatesFromString("bad, wrong")); + + Assertions.assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupStatesFromString(" bad, Stable")); + + Assertions.assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupStatesFromString(" , ,")); + } + + @Test + public void testConsumerGroupTypesFromString() { + Set result = ConsumerGroupCommand.consumerGroupTypesFromString("consumer"); + Assertions.assertEquals(ListConsumerGroupTest.set(Collections.singleton(GroupType.CONSUMER)), result); + + result = ConsumerGroupCommand.consumerGroupTypesFromString("consumer, classic"); + Assertions.assertEquals(ListConsumerGroupTest.set(Arrays.asList(GroupType.CONSUMER, GroupType.CLASSIC)), result); + + result = ConsumerGroupCommand.consumerGroupTypesFromString("Consumer, Classic"); + Assertions.assertEquals(ListConsumerGroupTest.set(Arrays.asList(GroupType.CONSUMER, GroupType.CLASSIC)), result); + + Assertions.assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString("bad, wrong")); + + Assertions.assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString(" bad, generic")); + + Assertions.assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString(" , ,")); + } +} \ No newline at end of file diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java index e86c61ee0ab37..c493a0c8ebcc8 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java @@ -17,36 +17,64 @@ package org.apache.kafka.tools.consumer.group; import joptsimple.OptionException; -import org.apache.kafka.clients.consumer.Consumer; +import kafka.test.ClusterConfig; +import kafka.test.ClusterInstance; +import kafka.test.annotation.ClusterTemplate; +import kafka.test.junit.ClusterTestExtensions; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.consumer.GroupProtocol; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.RangeAssignor; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.ConsumerGroupState; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import java.io.BufferedWriter; import java.io.File; import java.io.FileWriter; -import java.text.SimpleDateFormat; +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Calendar; -import java.util.Collection; -import java.util.Collections; -import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.Properties; -import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; +import static java.time.LocalDateTime.now; +import static java.util.Arrays.asList; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static java.util.Collections.singletonMap; +import static java.util.stream.Collectors.toMap; +import static org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.clients.admin.AdminClientConfig.RETRIES_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.ACKS_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG; import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -62,506 +90,775 @@ * - scope=topics+partitions, scenario=to-earliest * - export/import */ -public class ResetConsumerGroupOffsetTest extends ConsumerGroupCommandTest { - private String[] basicArgs() { +@ExtendWith(value = ClusterTestExtensions.class) +public class ResetConsumerGroupOffsetTest { + + private static final String TOPIC_PREFIX = "foo-"; + private static final String GROUP_PREFIX = "test.group-"; + + private static List generator() { + return ConsumerGroupCommandTestUtils.generator(); + } + + private String[] basicArgs(ClusterInstance cluster) { return new String[]{"--reset-offsets", - "--bootstrap-server", bootstrapServers(listenerName()), + "--bootstrap-server", cluster.bootstrapServers(), "--timeout", Long.toString(DEFAULT_MAX_WAIT_MS)}; } - private String[] buildArgsForGroups(List groups, String...args) { - List res = new ArrayList<>(Arrays.asList(basicArgs())); + private String[] buildArgsForGroups(ClusterInstance cluster, List groups, String... args) { + List res = new ArrayList<>(asList(basicArgs(cluster))); for (String group : groups) { res.add("--group"); res.add(group); } - res.addAll(Arrays.asList(args)); + res.addAll(asList(args)); return res.toArray(new String[0]); } - private String[] buildArgsForGroup(String group, String...args) { - return buildArgsForGroups(Collections.singletonList(group), args); + private String[] buildArgsForGroup(ClusterInstance cluster, String group, String... args) { + return buildArgsForGroups(cluster, singletonList(group), args); } - private String[] buildArgsForAllGroups(String...args) { - List res = new ArrayList<>(Arrays.asList(basicArgs())); + private String[] buildArgsForAllGroups(ClusterInstance cluster, String... args) { + List res = new ArrayList<>(asList(basicArgs(cluster))); res.add("--all-groups"); - res.addAll(Arrays.asList(args)); + res.addAll(asList(args)); return res.toArray(new String[0]); } - @Test - public void testResetOffsetsNotExistingGroup() throws Exception { + @ClusterTemplate("generator") + public void testResetOffsetsNotExistingGroup(ClusterInstance cluster) throws Exception { + String topic = generateRandomTopic(); String group = "missing.group"; - String[] args = buildArgsForGroup(group, "--all-topics", "--to-current", "--execute"); - ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args); - // Make sure we got a coordinator - TestUtils.waitForCondition( - () -> Objects.equals(consumerGroupCommand.collectGroupState(group).coordinator.host(), "localhost"), - "Can't find a coordinator"); - Map resetOffsets = consumerGroupCommand.resetOffsets().get(group); - assertTrue(resetOffsets.isEmpty()); - assertTrue(committedOffsets(TOPIC, group).isEmpty()); - } - - @Test - public void testResetOffsetsExistingTopic() { - String group = "new.group"; - String[] args = buildArgsForGroup(group, "--topic", TOPIC, "--to-offset", "50"); - produceMessages(TOPIC, 100); - resetAndAssertOffsets(args, 50, true, Collections.singletonList(TOPIC)); - resetAndAssertOffsets(addTo(args, "--dry-run"), 50, true, Collections.singletonList(TOPIC)); - resetAndAssertOffsets(addTo(args, "--execute"), 50, false, Collections.singletonList(TOPIC)); + String[] args = buildArgsForGroup(cluster, group, "--all-topics", "--to-current", "--execute"); + + try (ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) { + // Make sure we got a coordinator + TestUtils.waitForCondition( + () -> "localhost".equals(service.collectGroupState(group).coordinator.host()), + "Can't find a coordinator"); + Map resetOffsets = service.resetOffsets().get(group); + assertTrue(resetOffsets.isEmpty()); + assertTrue(committedOffsets(cluster, topic, group).isEmpty()); + } } - @Test - public void testResetOffsetsExistingTopicSelectedGroups() throws Exception { - produceMessages(TOPIC, 100); - List groups = IntStream.rangeClosed(1, 3).mapToObj(id -> GROUP + id).collect(Collectors.toList()); - for (String group : groups) { - ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, group, GroupProtocol.CLASSIC.name); - awaitConsumerProgress(TOPIC, group, 100L); - executor.shutdown(); - } - String[] args = buildArgsForGroups(groups, "--topic", TOPIC, "--to-offset", "50"); - resetAndAssertOffsets(args, 50, true, Collections.singletonList(TOPIC)); - resetAndAssertOffsets(addTo(args, "--dry-run"), 50, true, Collections.singletonList(TOPIC)); - resetAndAssertOffsets(addTo(args, "--execute"), 50, false, Collections.singletonList(TOPIC)); - } - - @Test - public void testResetOffsetsExistingTopicAllGroups() throws Exception { - String[] args = buildArgsForAllGroups("--topic", TOPIC, "--to-offset", "50"); - produceMessages(TOPIC, 100); - for (int i = 1; i <= 3; i++) { - String group = GROUP + i; - ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, group, GroupProtocol.CLASSIC.name); - awaitConsumerProgress(TOPIC, group, 100L); - executor.shutdown(); - } - resetAndAssertOffsets(args, 50, true, Collections.singletonList(TOPIC)); - resetAndAssertOffsets(addTo(args, "--dry-run"), 50, true, Collections.singletonList(TOPIC)); - resetAndAssertOffsets(addTo(args, "--execute"), 50, false, Collections.singletonList(TOPIC)); + @ClusterTemplate("generator") + public void testResetOffsetsExistingTopic(ClusterInstance cluster) { + String topic = generateRandomTopic(); + String group = "new.group"; + String[] args = buildArgsForGroup(cluster, group, "--topic", topic, "--to-offset", "50"); + + produceMessages(cluster, topic, 100); + resetAndAssertOffsets(cluster, args, 50, true, singletonList(topic)); + resetAndAssertOffsets(cluster, addTo(args, "--dry-run"), + 50, true, singletonList(topic)); + resetAndAssertOffsets(cluster, addTo(args, "--execute"), + 50, false, singletonList(topic)); } - @Test - public void testResetOffsetsAllTopicsAllGroups() throws Exception { - String[] args = buildArgsForAllGroups("--all-topics", "--to-offset", "50"); - List topics = IntStream.rangeClosed(1, 3).mapToObj(i -> TOPIC + i).collect(Collectors.toList()); - List groups = IntStream.rangeClosed(1, 3).mapToObj(i -> GROUP + i).collect(Collectors.toList()); - topics.forEach(topic -> produceMessages(topic, 100)); + @ClusterTemplate("generator") + public void testResetOffsetsExistingTopicSelectedGroups(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String topic = generateRandomTopic(); - for (String topic : topics) { + produceMessages(cluster, topic, 100); + List groups = generateIds(topic); for (String group : groups) { - ConsumerGroupExecutor executor = addConsumerGroupExecutor(3, topic, group, GroupProtocol.CLASSIC.name); - awaitConsumerProgress(topic, group, 100); - executor.shutdown(); + try (AutoCloseable consumerGroupCloseable = + consumerGroupClosable(cluster, 1, topic, group, groupProtocol)) { + awaitConsumerProgress(cluster, topic, group, 100L); + } } + + String[] args = buildArgsForGroups(cluster, groups, "--topic", topic, "--to-offset", "50"); + resetAndAssertOffsets(cluster, args, 50, true, singletonList(topic)); + resetAndAssertOffsets(cluster, addTo(args, "--dry-run"), + 50, true, singletonList(topic)); + resetAndAssertOffsets(cluster, addTo(args, "--execute"), + 50, false, singletonList(topic)); } - resetAndAssertOffsets(args, 50, true, topics); - resetAndAssertOffsets(addTo(args, "--dry-run"), 50, true, topics); - resetAndAssertOffsets(addTo(args, "--execute"), 50, false, topics); } - @Test - public void testResetOffsetsToLocalDateTime() throws Exception { - SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS"); - Calendar calendar = Calendar.getInstance(); - calendar.add(Calendar.DATE, -1); + @ClusterTemplate("generator") + public void testResetOffsetsExistingTopicAllGroups(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String topic = generateRandomTopic(); + String[] args = buildArgsForAllGroups(cluster, "--topic", topic, "--to-offset", "50"); - produceMessages(TOPIC, 100); + produceMessages(cluster, topic, 100); + for (int i = 1; i <= 3; i++) { + String group = generateRandomGroupId(); + try (AutoCloseable consumerGroupCloseable = + consumerGroupClosable(cluster, 1, topic, group, groupProtocol)) { + awaitConsumerProgress(cluster, topic, group, 100L); + } + } + resetAndAssertOffsets(cluster, args, 50, true, singletonList(topic)); + resetAndAssertOffsets(cluster, addTo(args, "--dry-run"), + 50, true, singletonList(topic)); + resetAndAssertOffsets(cluster, addTo(args, "--execute"), + 50, false, singletonList(topic)); + } + } + + @ClusterTemplate("generator") + public void testResetOffsetsAllTopicsAllGroups(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String groupId = generateRandomGroupId(); + String topicId = generateRandomTopic(); + + String[] args = buildArgsForAllGroups(cluster, "--all-topics", "--to-offset", "50"); + List topics = generateIds(groupId); + List groups = generateIds(topicId); + topics.forEach(topic -> produceMessages(cluster, topic, 100)); + + for (String topic : topics) { + for (String group : groups) { + try (AutoCloseable consumerGroupCloseable = + consumerGroupClosable(cluster, 3, topic, group, groupProtocol)) { + awaitConsumerProgress(cluster, topic, group, 100); + } + } + } - ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, GROUP, GroupProtocol.CLASSIC.name); - awaitConsumerProgress(TOPIC, GROUP, 100L); - executor.shutdown(); + resetAndAssertOffsets(cluster, args, 50, true, topics); + resetAndAssertOffsets(cluster, addTo(args, "--dry-run"), + 50, true, topics); + resetAndAssertOffsets(cluster, addTo(args, "--execute"), + 50, false, topics); - String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-datetime", format.format(calendar.getTime()), "--execute"); - resetAndAssertOffsets(args, 0); + try (Admin admin = cluster.createAdminClient()) { + admin.deleteConsumerGroups(groups).all().get(); + } + } } - @Test - public void testResetOffsetsToZonedDateTime() throws Exception { - SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"); + @ClusterTemplate("generator") + public void testResetOffsetsToLocalDateTime(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); - produceMessages(TOPIC, 50); - Date checkpoint = new Date(); - produceMessages(TOPIC, 50); + DateTimeFormatter format = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSS"); + LocalDateTime dateTime = now().minusDays(1); + String[] args = buildArgsForGroup(cluster, group, + "--all-topics", "--to-datetime", + format.format(dateTime), "--execute"); - ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, GROUP, GroupProtocol.CLASSIC.name); - awaitConsumerProgress(TOPIC, GROUP, 100L); - executor.shutdown(); + produceMessages(cluster, topic, 100); - String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-datetime", format.format(checkpoint), "--execute"); - resetAndAssertOffsets(args, 50); - } + try (AutoCloseable consumerGroupCloseable = + consumerGroupClosable(cluster, 1, topic, group, groupProtocol)) { + awaitConsumerProgress(cluster, topic, group, 100L); + } - @Test - public void testResetOffsetsByDuration() throws Exception { - String[] args = buildArgsForGroup(GROUP, "--all-topics", "--by-duration", "PT1M", "--execute"); - produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); - resetAndAssertOffsets(args, 0); + resetAndAssertOffsets(cluster, topic, args, 0); + } } - @Test - public void testResetOffsetsByDurationToEarliest() throws Exception { - String[] args = buildArgsForGroup(GROUP, "--all-topics", "--by-duration", "PT0.1S", "--execute"); - produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); - resetAndAssertOffsets(args, 100); - } + @ClusterTemplate("generator") + public void testResetOffsetsToZonedDateTime(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); + DateTimeFormatter format = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"); - @Test - public void testResetOffsetsByDurationFallbackToLatestWhenNoRecords() { - String topic = "foo2"; - String[] args = buildArgsForGroup(GROUP, "--topic", topic, "--by-duration", "PT1M", "--execute"); - createTopic(topic, 1, 1, new Properties(), listenerName(), new Properties()); - resetAndAssertOffsets(args, 0, false, Collections.singletonList("foo2")); + produceMessages(cluster, topic, 50); + ZonedDateTime checkpoint = now().atZone(ZoneId.systemDefault()); + produceMessages(cluster, topic, 50); - adminZkClient().deleteTopic(topic); - } + String[] args = buildArgsForGroup(cluster, group, + "--all-topics", "--to-datetime", format.format(checkpoint), + "--execute"); - @Test - public void testResetOffsetsToEarliest() throws Exception { - String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-earliest", "--execute"); - produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); - resetAndAssertOffsets(args, 0); - } + try (AutoCloseable consumerGroupCloseable = + consumerGroupClosable(cluster, 1, topic, group, groupProtocol)) { + awaitConsumerProgress(cluster, topic, group, 100L); + } - @Test - public void testResetOffsetsToLatest() throws Exception { - String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-latest", "--execute"); - produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); - produceMessages(TOPIC, 100); - resetAndAssertOffsets(args, 200); + resetAndAssertOffsets(cluster, topic, args, 50); + } } - @Test - public void testResetOffsetsToCurrentOffset() throws Exception { - String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-current", "--execute"); - produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); - produceMessages(TOPIC, 100); - resetAndAssertOffsets(args, 100); - } + @ClusterTemplate("generator") + public void testResetOffsetsByDuration(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); - @Test - public void testResetOffsetsToSpecificOffset() throws Exception { - String[] args = buildArgsForGroup(GROUP, "--all-topics", "--to-offset", "1", "--execute"); - produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); - resetAndAssertOffsets(args, 1); + String[] args = buildArgsForGroup(cluster, group, "--all-topics", "--by-duration", "PT1M", "--execute"); + produceConsumeAndShutdown(cluster, topic, group, 1, groupProtocol); + resetAndAssertOffsets(cluster, topic, args, 0); + } } - @Test - public void testResetOffsetsShiftPlus() throws Exception { - String[] args = buildArgsForGroup(GROUP, "--all-topics", "--shift-by", "50", "--execute"); - produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); - produceMessages(TOPIC, 100); - resetAndAssertOffsets(args, 150); - } + @ClusterTemplate("generator") + public void testResetOffsetsByDurationToEarliest(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); - @Test - public void testResetOffsetsShiftMinus() throws Exception { - String[] args = buildArgsForGroup(GROUP, "--all-topics", "--shift-by", "-50", "--execute"); - produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); - produceMessages(TOPIC, 100); - resetAndAssertOffsets(args, 50); + String[] args = buildArgsForGroup(cluster, group, "--all-topics", "--by-duration", "PT0.1S", "--execute"); + produceConsumeAndShutdown(cluster, topic, group, 1, groupProtocol); + resetAndAssertOffsets(cluster, topic, args, 100); + } } - @Test - public void testResetOffsetsShiftByLowerThanEarliest() throws Exception { - String[] args = buildArgsForGroup(GROUP, "--all-topics", "--shift-by", "-150", "--execute"); - produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); - produceMessages(TOPIC, 100); - resetAndAssertOffsets(args, 0); - } + @ClusterTemplate("generator") + public void testResetOffsetsByDurationFallbackToLatestWhenNoRecords(ClusterInstance cluster) throws ExecutionException, InterruptedException { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); + + String[] args = buildArgsForGroup(cluster, group, "--topic", topic, "--by-duration", "PT1M", "--execute"); - @Test - public void testResetOffsetsShiftByHigherThanLatest() throws Exception { - String[] args = buildArgsForGroup(GROUP, "--all-topics", "--shift-by", "150", "--execute"); - produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); - produceMessages(TOPIC, 100); - resetAndAssertOffsets(args, 200); + try (Admin admin = cluster.createAdminClient()) { + admin.createTopics(singleton(new NewTopic(topic, 1, (short) 1))).all().get(); + resetAndAssertOffsets(cluster, args, 0, false, singletonList(topic)); + admin.deleteTopics(singleton(topic)).all().get(); + } } - @Test - public void testResetOffsetsToEarliestOnOneTopic() throws Exception { - String[] args = buildArgsForGroup(GROUP, "--topic", TOPIC, "--to-earliest", "--execute"); - produceConsumeAndShutdown(TOPIC, GROUP, 100, 1); - resetAndAssertOffsets(args, 0); + @ClusterTemplate("generator") + public void testResetOffsetsToEarliest(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); + + String[] args = buildArgsForGroup(cluster, group, "--all-topics", "--to-earliest", "--execute"); + produceConsumeAndShutdown(cluster, topic, group, 1, groupProtocol); + resetAndAssertOffsets(cluster, topic, args, 0); + } } - @Test - public void testResetOffsetsToEarliestOnOneTopicAndPartition() throws Exception { - String topic = "bar"; - createTopic(topic, 2, 1, new Properties(), listenerName(), new Properties()); + @ClusterTemplate("generator") + public void testResetOffsetsToLatest(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); + + String[] args = buildArgsForGroup(cluster, group, "--all-topics", "--to-latest", "--execute"); + produceConsumeAndShutdown(cluster, topic, group, 1, groupProtocol); + produceMessages(cluster, topic, 100); + resetAndAssertOffsets(cluster, topic, args, 200); + } + } - String[] args = buildArgsForGroup(GROUP, "--topic", topic + ":1", "--to-earliest", "--execute"); - ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args); + @ClusterTemplate("generator") + public void testResetOffsetsToCurrentOffset(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); - produceConsumeAndShutdown(topic, GROUP, 100, 2); - Map priorCommittedOffsets = committedOffsets(topic, GROUP); + String[] args = buildArgsForGroup(cluster, group, "--all-topics", "--to-current", "--execute"); + produceConsumeAndShutdown(cluster, topic, group, 1, groupProtocol); + produceMessages(cluster, topic, 100); + resetAndAssertOffsets(cluster, topic, args, 100); + } + } - TopicPartition tp0 = new TopicPartition(topic, 0); - TopicPartition tp1 = new TopicPartition(topic, 1); - Map expectedOffsets = new HashMap<>(); - expectedOffsets.put(tp0, priorCommittedOffsets.get(tp0)); - expectedOffsets.put(tp1, 0L); - resetAndAssertOffsetsCommitted(consumerGroupCommand, expectedOffsets, topic); + @ClusterTemplate("generator") + public void testResetOffsetsToSpecificOffset(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); - adminZkClient().deleteTopic(topic); + String[] args = buildArgsForGroup(cluster, group, "--all-topics", "--to-offset", "1", "--execute"); + produceConsumeAndShutdown(cluster, topic, group, 1, groupProtocol); + resetAndAssertOffsets(cluster, topic, args, 1); + } } - @Test - public void testResetOffsetsToEarliestOnTopics() throws Exception { - String topic1 = "topic1"; - String topic2 = "topic2"; - createTopic(topic1, 1, 1, new Properties(), listenerName(), new Properties()); - createTopic(topic2, 1, 1, new Properties(), listenerName(), new Properties()); + @ClusterTemplate("generator") + public void testResetOffsetsShiftPlus(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); - String[] args = buildArgsForGroup(GROUP, "--topic", topic1, "--topic", topic2, "--to-earliest", "--execute"); - ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args); + String[] args = buildArgsForGroup(cluster, group, "--all-topics", "--shift-by", "50", "--execute"); + produceConsumeAndShutdown(cluster, topic, group, 1, groupProtocol); + produceMessages(cluster, topic, 100); + resetAndAssertOffsets(cluster, topic, args, 150); + } + } - produceConsumeAndShutdown(topic1, GROUP, 100, 1); - produceConsumeAndShutdown(topic2, GROUP, 100, 1); + @ClusterTemplate("generator") + public void testResetOffsetsShiftMinus(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); - TopicPartition tp1 = new TopicPartition(topic1, 0); - TopicPartition tp2 = new TopicPartition(topic2, 0); + String[] args = buildArgsForGroup(cluster, group, "--all-topics", "--shift-by", "-50", "--execute"); + produceConsumeAndShutdown(cluster, topic, group, 1, groupProtocol); + produceMessages(cluster, topic, 100); + resetAndAssertOffsets(cluster, topic, args, 50); + } + } - Map allResetOffsets = toOffsetMap(resetOffsets(consumerGroupCommand).get(GROUP)); - Map expMap = new HashMap<>(); - expMap.put(tp1, 0L); - expMap.put(tp2, 0L); - assertEquals(expMap, allResetOffsets); - assertEquals(Collections.singletonMap(tp1, 0L), committedOffsets(topic1, GROUP)); - assertEquals(Collections.singletonMap(tp2, 0L), committedOffsets(topic2, GROUP)); + @ClusterTemplate("generator") + public void testResetOffsetsShiftByLowerThanEarliest(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); - adminZkClient().deleteTopic(topic1); - adminZkClient().deleteTopic(topic2); + String[] args = buildArgsForGroup(cluster, group, "--all-topics", "--shift-by", "-150", "--execute"); + produceConsumeAndShutdown(cluster, topic, group, 1, groupProtocol); + produceMessages(cluster, topic, 100); + resetAndAssertOffsets(cluster, topic, args, 0); + } } - @Test - public void testResetOffsetsToEarliestOnTopicsAndPartitions() throws Exception { - String topic1 = "topic1"; - String topic2 = "topic2"; + @ClusterTemplate("generator") + public void testResetOffsetsShiftByHigherThanLatest(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); - createTopic(topic1, 2, 1, new Properties(), listenerName(), new Properties()); - createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties()); + String[] args = buildArgsForGroup(cluster, group, "--all-topics", "--shift-by", "150", "--execute"); + produceConsumeAndShutdown(cluster, topic, group, 1, groupProtocol); + produceMessages(cluster, topic, 100); + resetAndAssertOffsets(cluster, topic, args, 200); + } + } - String[] args = buildArgsForGroup(GROUP, "--topic", topic1 + ":1", "--topic", topic2 + ":1", "--to-earliest", "--execute"); - ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args); + @ClusterTemplate("generator") + public void testResetOffsetsToEarliestOnOneTopic(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); - produceConsumeAndShutdown(topic1, GROUP, 100, 2); - produceConsumeAndShutdown(topic2, GROUP, 100, 2); + String[] args = buildArgsForGroup(cluster, group, "--topic", topic, "--to-earliest", "--execute"); + produceConsumeAndShutdown(cluster, topic, group, 1, groupProtocol); + resetAndAssertOffsets(cluster, topic, args, 0); + } + } - Map priorCommittedOffsets1 = committedOffsets(topic1, GROUP); - Map priorCommittedOffsets2 = committedOffsets(topic2, GROUP); + @ClusterTemplate("generator") + public void testResetOffsetsToEarliestOnOneTopicAndPartition(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); + String[] args = buildArgsForGroup(cluster, group, "--topic", topic + ":1", + "--to-earliest", "--execute"); + + try (Admin admin = cluster.createAdminClient(); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) { + admin.createTopics(singleton(new NewTopic(topic, 2, (short) 1))).all().get(); + + produceConsumeAndShutdown(cluster, topic, group, 2, groupProtocol); + Map priorCommittedOffsets = committedOffsets(cluster, topic, group); + TopicPartition tp0 = new TopicPartition(topic, 0); + TopicPartition tp1 = new TopicPartition(topic, 1); + Map expectedOffsets = new HashMap<>(); + expectedOffsets.put(tp0, priorCommittedOffsets.get(tp0)); + expectedOffsets.put(tp1, 0L); + resetAndAssertOffsetsCommitted(cluster, service, expectedOffsets, topic); + + admin.deleteTopics(singleton(topic)).all().get(); + } + } + } - TopicPartition tp1 = new TopicPartition(topic1, 1); - TopicPartition tp2 = new TopicPartition(topic2, 1); - Map allResetOffsets = toOffsetMap(resetOffsets(consumerGroupCommand).get(GROUP)); - Map expMap = new HashMap<>(); - expMap.put(tp1, 0L); - expMap.put(tp2, 0L); - assertEquals(expMap, allResetOffsets); - priorCommittedOffsets1.put(tp1, 0L); - assertEquals(priorCommittedOffsets1, committedOffsets(topic1, GROUP)); - priorCommittedOffsets2.put(tp2, 0L); - assertEquals(priorCommittedOffsets2, committedOffsets(topic2, GROUP)); + @ClusterTemplate("generator") + public void testResetOffsetsToEarliestOnTopics(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic1 = generateRandomTopic(); + String topic2 = generateRandomTopic(); + String[] args = buildArgsForGroup(cluster, group, + "--topic", topic1, + "--topic", topic2, + "--to-earliest", "--execute"); + + try (Admin admin = cluster.createAdminClient(); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) { + admin.createTopics(asList(new NewTopic(topic1, 1, (short) 1), + new NewTopic(topic2, 1, (short) 1))).all().get(); + + produceConsumeAndShutdown(cluster, topic1, group, 1, groupProtocol); + produceConsumeAndShutdown(cluster, topic2, group, 1, groupProtocol); + + TopicPartition tp1 = new TopicPartition(topic1, 0); + TopicPartition tp2 = new TopicPartition(topic2, 0); + + Map allResetOffsets = toOffsetMap(resetOffsets(service).get(group)); + Map expMap = new HashMap<>(); + expMap.put(tp1, 0L); + expMap.put(tp2, 0L); + assertEquals(expMap, allResetOffsets); + assertEquals(singletonMap(tp1, 0L), committedOffsets(cluster, topic1, group)); + assertEquals(singletonMap(tp2, 0L), committedOffsets(cluster, topic2, group)); + + admin.deleteTopics(asList(topic1, topic2)).all().get(); + } + } + } - adminZkClient().deleteTopic(topic1); - adminZkClient().deleteTopic(topic2); + @ClusterTemplate("generator") + public void testResetOffsetsToEarliestOnTopicsAndPartitions(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic1 = generateRandomTopic(); + String topic2 = generateRandomTopic(); + String[] args = buildArgsForGroup(cluster, group, + "--topic", topic1 + ":1", + "--topic", topic2 + ":1", + "--to-earliest", "--execute"); + + try (Admin admin = cluster.createAdminClient(); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) { + admin.createTopics(asList(new NewTopic(topic1, 2, (short) 1), + new NewTopic(topic2, 2, (short) 1))).all().get(); + + produceConsumeAndShutdown(cluster, topic1, group, 2, groupProtocol); + produceConsumeAndShutdown(cluster, topic2, group, 2, groupProtocol); + + Map priorCommittedOffsets1 = + committedOffsets(cluster, topic1, group); + Map priorCommittedOffsets2 = + committedOffsets(cluster, topic2, group); + + TopicPartition tp1 = new TopicPartition(topic1, 1); + TopicPartition tp2 = new TopicPartition(topic2, 1); + Map allResetOffsets = toOffsetMap(resetOffsets(service).get(group)); + Map expMap = new HashMap<>(); + expMap.put(tp1, 0L); + expMap.put(tp2, 0L); + assertEquals(expMap, allResetOffsets); + priorCommittedOffsets1.put(tp1, 0L); + assertEquals(priorCommittedOffsets1, committedOffsets(cluster, topic1, group)); + priorCommittedOffsets2.put(tp2, 0L); + assertEquals(priorCommittedOffsets2, committedOffsets(cluster, topic2, group)); + + admin.deleteTopics(asList(topic1, topic2)).all().get(); + } + } } - @Test - // This one deals with old CSV export/import format for a single --group arg: "topic,partition,offset" to support old behavior - public void testResetOffsetsExportImportPlanSingleGroupArg() throws Exception { - String topic = "bar"; - TopicPartition tp0 = new TopicPartition(topic, 0); - TopicPartition tp1 = new TopicPartition(topic, 1); - createTopic(topic, 2, 1, new Properties(), listenerName(), new Properties()); + @ClusterTemplate("generator") + // This one deals with old CSV export/import format for a single --group arg: + // "topic,partition,offset" to support old behavior + public void testResetOffsetsExportImportPlanSingleGroupArg(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group = generateRandomGroupId(); + String topic = generateRandomTopic(); + + TopicPartition tp0 = new TopicPartition(topic, 0); + TopicPartition tp1 = new TopicPartition(topic, 1); + String[] cgcArgs = buildArgsForGroup(cluster, group, "--all-topics", "--to-offset", "2", "--export"); + File file = TestUtils.tempFile("reset", ".csv"); - String[] cgcArgs = buildArgsForGroup(GROUP, "--all-topics", "--to-offset", "2", "--export"); - ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(cgcArgs); + try (Admin admin = cluster.createAdminClient(); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs)) { - produceConsumeAndShutdown(topic, GROUP, 100, 2); + admin.createTopics(singleton(new NewTopic(topic, 2, (short) 1))).all().get(); + produceConsumeAndShutdown(cluster, topic, group, 2, groupProtocol); - File file = TestUtils.tempFile("reset", ".csv"); + Map> exportedOffsets = service.resetOffsets(); - Map> exportedOffsets = consumerGroupCommand.resetOffsets(); - BufferedWriter bw = new BufferedWriter(new FileWriter(file)); - bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets)); - bw.close(); + writeContentToFile(file, service.exportOffsetsToCsv(exportedOffsets)); - Map exp1 = new HashMap<>(); - exp1.put(tp0, 2L); - exp1.put(tp1, 2L); - assertEquals(exp1, toOffsetMap(exportedOffsets.get(GROUP))); + Map exp1 = new HashMap<>(); + exp1.put(tp0, 2L); + exp1.put(tp1, 2L); + assertEquals(exp1, toOffsetMap(exportedOffsets.get(group))); - String[] cgcArgsExec = buildArgsForGroup(GROUP, "--all-topics", "--from-file", file.getCanonicalPath(), "--dry-run"); - ConsumerGroupCommand.ConsumerGroupService consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec); - Map> importedOffsets = consumerGroupCommandExec.resetOffsets(); - assertEquals(exp1, toOffsetMap(importedOffsets.get(GROUP))); + String[] cgcArgsExec = buildArgsForGroup(cluster, group, "--all-topics", + "--from-file", file.getCanonicalPath(), "--dry-run"); + try (ConsumerGroupCommand.ConsumerGroupService serviceExec = getConsumerGroupService(cgcArgsExec)) { + Map> importedOffsets = serviceExec.resetOffsets(); + assertEquals(exp1, toOffsetMap(importedOffsets.get(group))); + } - adminZkClient().deleteTopic(topic); + admin.deleteTopics(singleton(topic)); + } + } } - @Test + @ClusterTemplate("generator") // This one deals with universal CSV export/import file format "group,topic,partition,offset", // supporting multiple --group args or --all-groups arg - public void testResetOffsetsExportImportPlan() throws Exception { - String group1 = GROUP + "1"; - String group2 = GROUP + "2"; - String topic1 = "bar1"; - String topic2 = "bar2"; - TopicPartition t1p0 = new TopicPartition(topic1, 0); - TopicPartition t1p1 = new TopicPartition(topic1, 1); - TopicPartition t2p0 = new TopicPartition(topic2, 0); - TopicPartition t2p1 = new TopicPartition(topic2, 1); - createTopic(topic1, 2, 1, new Properties(), listenerName(), new Properties()); - createTopic(topic2, 2, 1, new Properties(), listenerName(), new Properties()); - - String[] cgcArgs = buildArgsForGroups(Arrays.asList(group1, group2), "--all-topics", "--to-offset", "2", "--export"); - ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(cgcArgs); - - produceConsumeAndShutdown(topic1, group1, 100, 1); - produceConsumeAndShutdown(topic2, group2, 100, 1); - - awaitConsumerGroupInactive(consumerGroupCommand, group1); - awaitConsumerGroupInactive(consumerGroupCommand, group2); - - File file = TestUtils.tempFile("reset", ".csv"); - - Map> exportedOffsets = consumerGroupCommand.resetOffsets(); - BufferedWriter bw = new BufferedWriter(new FileWriter(file)); - bw.write(consumerGroupCommand.exportOffsetsToCsv(exportedOffsets)); - bw.close(); - Map exp1 = new HashMap<>(); - exp1.put(t1p0, 2L); - exp1.put(t1p1, 2L); - Map exp2 = new HashMap<>(); - exp2.put(t2p0, 2L); - exp2.put(t2p1, 2L); - - assertEquals(exp1, toOffsetMap(exportedOffsets.get(group1))); - assertEquals(exp2, toOffsetMap(exportedOffsets.get(group2))); - - // Multiple --group's offset import - String[] cgcArgsExec = buildArgsForGroups(Arrays.asList(group1, group2), "--all-topics", "--from-file", file.getCanonicalPath(), "--dry-run"); - ConsumerGroupCommand.ConsumerGroupService consumerGroupCommandExec = getConsumerGroupService(cgcArgsExec); - Map> importedOffsets = consumerGroupCommandExec.resetOffsets(); - assertEquals(exp1, toOffsetMap(importedOffsets.get(group1))); - assertEquals(exp2, toOffsetMap(importedOffsets.get(group2))); - - // Single --group offset import using "group,topic,partition,offset" csv format - String[] cgcArgsExec2 = buildArgsForGroup(group1, "--all-topics", "--from-file", file.getCanonicalPath(), "--dry-run"); - ConsumerGroupCommand.ConsumerGroupService consumerGroupCommandExec2 = getConsumerGroupService(cgcArgsExec2); - Map> importedOffsets2 = consumerGroupCommandExec2.resetOffsets(); - assertEquals(exp1, toOffsetMap(importedOffsets2.get(group1))); - - adminZkClient().deleteTopic(TOPIC); - } - - @Test - public void testResetWithUnrecognizedNewConsumerOption() { - String[] cgcArgs = new String[]{"--new-consumer", "--bootstrap-server", bootstrapServers(listenerName()), "--reset-offsets", - "--group", GROUP, "--all-topics", "--to-offset", "2", "--export"}; + public void testResetOffsetsExportImportPlan(ClusterInstance cluster) throws Exception { + for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) { + String group1 = generateRandomGroupId(); + String group2 = generateRandomGroupId(); + String topic1 = generateRandomTopic(); + String topic2 = generateRandomTopic(); + + TopicPartition t1p0 = new TopicPartition(topic1, 0); + TopicPartition t1p1 = new TopicPartition(topic1, 1); + TopicPartition t2p0 = new TopicPartition(topic2, 0); + TopicPartition t2p1 = new TopicPartition(topic2, 1); + String[] cgcArgs = buildArgsForGroups(cluster, asList(group1, group2), + "--all-topics", "--to-offset", "2", "--export"); + File file = TestUtils.tempFile("reset", ".csv"); + + try (Admin admin = cluster.createAdminClient(); + ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs)) { + + admin.createTopics(asList(new NewTopic(topic1, 2, (short) 1), + new NewTopic(topic2, 2, (short) 1))).all().get(); + + produceConsumeAndShutdown(cluster, topic1, group1, 1, groupProtocol); + produceConsumeAndShutdown(cluster, topic2, group2, 1, groupProtocol); + + awaitConsumerGroupInactive(service, group1); + awaitConsumerGroupInactive(service, group2); + + Map> exportedOffsets = service.resetOffsets(); + + writeContentToFile(file, service.exportOffsetsToCsv(exportedOffsets)); + + Map exp1 = new HashMap<>(); + exp1.put(t1p0, 2L); + exp1.put(t1p1, 2L); + Map exp2 = new HashMap<>(); + exp2.put(t2p0, 2L); + exp2.put(t2p1, 2L); + + assertEquals(exp1, toOffsetMap(exportedOffsets.get(group1))); + assertEquals(exp2, toOffsetMap(exportedOffsets.get(group2))); + + // Multiple --group's offset import + String[] cgcArgsExec = buildArgsForGroups(cluster, asList(group1, group2), + "--all-topics", + "--from-file", file.getCanonicalPath(), "--dry-run"); + try (ConsumerGroupCommand.ConsumerGroupService serviceExec = getConsumerGroupService(cgcArgsExec)) { + Map> importedOffsets = serviceExec.resetOffsets(); + assertEquals(exp1, toOffsetMap(importedOffsets.get(group1))); + assertEquals(exp2, toOffsetMap(importedOffsets.get(group2))); + } + + // Single --group offset import using "group,topic,partition,offset" csv format + String[] cgcArgsExec2 = buildArgsForGroup(cluster, group1, "--all-topics", + "--from-file", file.getCanonicalPath(), "--dry-run"); + try (ConsumerGroupCommand.ConsumerGroupService serviceExec2 = getConsumerGroupService(cgcArgsExec2)) { + Map> importedOffsets2 = serviceExec2.resetOffsets(); + assertEquals(exp1, toOffsetMap(importedOffsets2.get(group1))); + } + + admin.deleteTopics(asList(topic1, topic2)); + } + } + } + + @ClusterTemplate("generator") + public void testResetWithUnrecognizedNewConsumerOption(ClusterInstance cluster) { + String group = generateRandomGroupId(); + String[] cgcArgs = new String[]{"--new-consumer", + "--bootstrap-server", cluster.bootstrapServers(), + "--reset-offsets", "--group", group, "--all-topics", + "--to-offset", "2", "--export"}; assertThrows(OptionException.class, () -> getConsumerGroupService(cgcArgs)); } - private void produceMessages(String topic, int numMessages) { - List> records = IntStream.range(0, numMessages) - .mapToObj(i -> new ProducerRecord(topic, new byte[100 * 1000])) - .collect(Collectors.toList()); - kafka.utils.TestUtils.produceMessages(servers(), seq(records), 1); + private String generateRandomTopic() { + return TOPIC_PREFIX + TestUtils.randomString(10); } - private void produceConsumeAndShutdown(String topic, String group, int totalMessages, int numConsumers) throws Exception { - produceMessages(topic, totalMessages); - ConsumerGroupExecutor executor = addConsumerGroupExecutor(numConsumers, topic, group, GroupProtocol.CLASSIC.name); - awaitConsumerProgress(topic, group, totalMessages); - executor.shutdown(); + private String generateRandomGroupId() { + return GROUP_PREFIX + TestUtils.randomString(10); } - private void awaitConsumerProgress(String topic, - String group, - long count) throws Exception { - try (Consumer consumer = createNoAutoCommitConsumer(group)) { - Set partitions = consumer.partitionsFor(topic).stream() - .map(partitionInfo -> new TopicPartition(partitionInfo.topic(), partitionInfo.partition())) - .collect(Collectors.toSet()); - - TestUtils.waitForCondition(() -> { - Collection committed = consumer.committed(partitions).values(); - long total = committed.stream() - .mapToLong(offsetAndMetadata -> Optional.ofNullable(offsetAndMetadata).map(OffsetAndMetadata::offset).orElse(0L)) - .sum(); - - return total == count; - }, "Expected that consumer group has consumed all messages from topic/partition. " + - "Expected offset: " + count + ". Actual offset: " + committedOffsets(topic, group).values().stream().mapToLong(Long::longValue).sum()); + private Map committedOffsets(ClusterInstance cluster, + String topic, + String group) { + try (Admin admin = Admin.create(singletonMap(BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()))) { + return admin.listConsumerGroupOffsets(group) + .all().get() + .get(group).entrySet() + .stream() + .filter(e -> e.getKey().topic().equals(topic)) + .collect(toMap(Map.Entry::getKey, e -> e.getValue().offset())); + } catch (ExecutionException | InterruptedException e) { + throw new RuntimeException(e); } } - private void awaitConsumerGroupInactive(ConsumerGroupCommand.ConsumerGroupService consumerGroupService, String group) throws Exception { - TestUtils.waitForCondition(() -> { - ConsumerGroupState state = consumerGroupService.collectGroupState(group).state; - return Objects.equals(state, ConsumerGroupState.EMPTY) || Objects.equals(state, ConsumerGroupState.DEAD); - }, "Expected that consumer group is inactive. Actual state: " + consumerGroupService.collectGroupState(group).state); + private ConsumerGroupCommand.ConsumerGroupService getConsumerGroupService(String[] args) { + return new ConsumerGroupCommand.ConsumerGroupService( + ConsumerGroupCommandOptions.fromArgs(args), + singletonMap(RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE))); } - private void resetAndAssertOffsets(String[] args, - long expectedOffset) { - resetAndAssertOffsets(args, expectedOffset, false, Collections.singletonList(TOPIC)); + private void produceMessages(ClusterInstance cluster, String topic, int numMessages) { + List> records = IntStream.range(0, numMessages) + .mapToObj(i -> new ProducerRecord(topic, new byte[100 * 1000])) + .collect(Collectors.toList()); + produceMessages(cluster, records); } - private void resetAndAssertOffsets(String[] args, - long expectedOffset, - boolean dryRun, - List topics) { - ConsumerGroupCommand.ConsumerGroupService consumerGroupCommand = getConsumerGroupService(args); - Map> expectedOffsets = topics.stream().collect(Collectors.toMap( - Function.identity(), - topic -> Collections.singletonMap(new TopicPartition(topic, 0), expectedOffset))); - Map> resetOffsetsResultByGroup = resetOffsets(consumerGroupCommand); + private void produceMessages(ClusterInstance cluster, List> records) { + try (Producer producer = createProducer(cluster)) { + records.forEach(producer::send); + } + } + + private Producer createProducer(ClusterInstance cluster) { + Properties props = new Properties(); + props.put(BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + props.put(ACKS_CONFIG, "1"); + props.put(KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + props.put(VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + return new KafkaProducer<>(props); + } - try { + private void resetAndAssertOffsets(ClusterInstance cluster, + String topic, + String[] args, + long expectedOffset) { + resetAndAssertOffsets(cluster, args, expectedOffset, false, singletonList(topic)); + } + + private void resetAndAssertOffsets(ClusterInstance cluster, + String[] args, + long expectedOffset, + boolean dryRun, + List topics) { + try (ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) { + Map> topicToExpectedOffsets = getTopicExceptOffsets(topics, expectedOffset); + Map> resetOffsetsResultByGroup = + resetOffsets(service); for (final String topic : topics) { resetOffsetsResultByGroup.forEach((group, partitionInfo) -> { - Map priorOffsets = committedOffsets(topic, group); - assertEquals(expectedOffsets.get(topic), - partitionInfo.entrySet().stream() - .filter(entry -> Objects.equals(entry.getKey().topic(), topic)) - .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset()))); - assertEquals(dryRun ? priorOffsets : expectedOffsets.get(topic), committedOffsets(topic, group)); + Map priorOffsets = committedOffsets(cluster, topic, group); + assertEquals(topicToExpectedOffsets.get(topic), partitionToOffsets(topic, partitionInfo)); + assertEquals(dryRun ? priorOffsets : topicToExpectedOffsets.get(topic), + committedOffsets(cluster, topic, group)); }); } - } finally { - consumerGroupCommand.close(); } } - private void resetAndAssertOffsetsCommitted(ConsumerGroupCommand.ConsumerGroupService consumerGroupService, - Map expectedOffsets, - String topic) { - Map> allResetOffsets = resetOffsets(consumerGroupService); - - allResetOffsets.forEach((group, offsetsInfo) -> { - offsetsInfo.forEach((tp, offsetMetadata) -> { - assertEquals(offsetMetadata.offset(), expectedOffsets.get(tp)); - assertEquals(expectedOffsets, committedOffsets(topic, group)); - }); - }); + private Map> getTopicExceptOffsets(List topics, + long expectedOffset) { + return topics.stream() + .collect(toMap(Function.identity(), + topic -> singletonMap(new TopicPartition(topic, 0), + expectedOffset))); } - private Map> resetOffsets(ConsumerGroupCommand.ConsumerGroupService consumerGroupService) { + private Map> resetOffsets( + ConsumerGroupCommand.ConsumerGroupService consumerGroupService) { return consumerGroupService.resetOffsets(); } - Map toOffsetMap(Map map) { - return map.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().offset())); + private Map partitionToOffsets(String topic, + Map partitionInfo) { + return partitionInfo.entrySet() + .stream() + .filter(entry -> Objects.equals(entry.getKey().topic(), topic)) + .collect(toMap(Map.Entry::getKey, e -> e.getValue().offset())); + } + + private static List generateIds(String name) { + return IntStream.rangeClosed(1, 2) + .mapToObj(id -> name + id) + .collect(Collectors.toList()); + } + + private void produceConsumeAndShutdown(ClusterInstance cluster, + String topic, + String group, + int numConsumers, + GroupProtocol groupProtocol) throws Exception { + produceMessages(cluster, topic, 100); + try (AutoCloseable consumerGroupCloseable = + consumerGroupClosable(cluster, numConsumers, topic, group, groupProtocol)) { + awaitConsumerProgress(cluster, topic, group, 100); + } + } + + private void writeContentToFile(File file, String content) throws IOException { + try (BufferedWriter bw = new BufferedWriter(new FileWriter(file))) { + bw.write(content); + } + } + + private AutoCloseable consumerGroupClosable(ClusterInstance cluster, + int numConsumers, + String topic, + String group, + GroupProtocol groupProtocol) { + Map configs = composeConsumerConfigs(cluster, group, groupProtocol); + return ConsumerGroupCommandTestUtils.buildConsumers( + numConsumers, + false, + topic, + () -> new KafkaConsumer(configs)); + } + + private Map composeConsumerConfigs(ClusterInstance cluster, + String group, + GroupProtocol groupProtocol) { + HashMap configs = new HashMap<>(); + configs.put(BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + configs.put(GROUP_ID_CONFIG, group); + configs.put(GROUP_PROTOCOL_CONFIG, groupProtocol.name); + configs.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + configs.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + configs.put(PARTITION_ASSIGNMENT_STRATEGY_CONFIG, RangeAssignor.class.getName()); + configs.put(AUTO_COMMIT_INTERVAL_MS_CONFIG, 1000); + configs.put(GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, 1000); + return configs; + } + + private void awaitConsumerProgress(ClusterInstance cluster, + String topic, + String group, + long count) throws Exception { + try (Admin admin = Admin.create(singletonMap(BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()))) { + Supplier offsets = () -> { + try { + return admin.listConsumerGroupOffsets(group) + .all().get().get(group) + .entrySet() + .stream() + .filter(e -> e.getKey().topic().equals(topic)) + .mapToLong(e -> e.getValue().offset()) + .sum(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + }; + TestUtils.waitForCondition(() -> offsets.get() == count, + "Expected that consumer group has consumed all messages from topic/partition. " + + "Expected offset: " + count + + ". Actual offset: " + offsets.get()); + } + } + + private void awaitConsumerGroupInactive(ConsumerGroupCommand.ConsumerGroupService service, + String group) throws Exception { + TestUtils.waitForCondition(() -> { + ConsumerGroupState state = service.collectGroupState(group).state; + return Objects.equals(state, ConsumerGroupState.EMPTY) || Objects.equals(state, ConsumerGroupState.DEAD); + }, "Expected that consumer group is inactive. Actual state: " + + service.collectGroupState(group).state); + } + + private void resetAndAssertOffsetsCommitted(ClusterInstance cluster, + ConsumerGroupCommand.ConsumerGroupService service, + Map expectedOffsets, + String topic) { + Map> allResetOffsets = resetOffsets(service); + + allResetOffsets.forEach((group, offsetsInfo) -> offsetsInfo.forEach((tp, offsetMetadata) -> { + assertEquals(offsetMetadata.offset(), expectedOffsets.get(tp)); + assertEquals(expectedOffsets, committedOffsets(cluster, topic, group)); + })); + } + + private Map toOffsetMap(Map map) { + return map.entrySet() + .stream() + .collect(toMap(Map.Entry::getKey, e -> e.getValue().offset())); } - private String[] addTo(String[] args, String...extra) { - List res = new ArrayList<>(Arrays.asList(args)); - res.addAll(Arrays.asList(extra)); + private String[] addTo(String[] args, String... extra) { + List res = new ArrayList<>(asList(args)); + res.addAll(asList(extra)); return res.toArray(new String[0]); } } diff --git a/tools/tools-api/src/main/java/org/apache/kafka/tools/api/Decoder.java b/tools/tools-api/src/main/java/org/apache/kafka/tools/api/Decoder.java new file mode 100644 index 0000000000000..f3bb9eb47fb58 --- /dev/null +++ b/tools/tools-api/src/main/java/org/apache/kafka/tools/api/Decoder.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tools.api; + +/** + * A decoder is a method of turning byte arrays into objects. + */ +@FunctionalInterface +public interface Decoder { + T fromBytes(byte[] bytes); +} diff --git a/tools/tools-api/src/main/java/org/apache/kafka/tools/api/DefaultDecoder.java b/tools/tools-api/src/main/java/org/apache/kafka/tools/api/DefaultDecoder.java new file mode 100644 index 0000000000000..e72aaac2e6c95 --- /dev/null +++ b/tools/tools-api/src/main/java/org/apache/kafka/tools/api/DefaultDecoder.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tools.api; + +/** + * The default implementation does nothing, just returns the same byte array it takes in. + */ +public class DefaultDecoder implements Decoder { + @Override + public byte[] fromBytes(byte[] bytes) { + return bytes; + } +} diff --git a/tools/tools-api/src/main/java/org/apache/kafka/tools/api/IntegerDecoder.java b/tools/tools-api/src/main/java/org/apache/kafka/tools/api/IntegerDecoder.java new file mode 100644 index 0000000000000..7e20140927298 --- /dev/null +++ b/tools/tools-api/src/main/java/org/apache/kafka/tools/api/IntegerDecoder.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tools.api; + +import java.nio.ByteBuffer; + +/** + * The integer decoder translates bytes into integers. + */ +public class IntegerDecoder implements Decoder { + @Override + public Integer fromBytes(byte[] bytes) { + return ByteBuffer.wrap(bytes).getInt(); + } +} diff --git a/tools/tools-api/src/main/java/org/apache/kafka/tools/api/LongDecoder.java b/tools/tools-api/src/main/java/org/apache/kafka/tools/api/LongDecoder.java new file mode 100644 index 0000000000000..8cdb4bbf6bb36 --- /dev/null +++ b/tools/tools-api/src/main/java/org/apache/kafka/tools/api/LongDecoder.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tools.api; + +import java.nio.ByteBuffer; + +/** + * The long decoder translates bytes into longs. + */ +public class LongDecoder implements Decoder { + @Override + public Long fromBytes(byte[] bytes) { + return ByteBuffer.wrap(bytes).getLong(); + } +} diff --git a/tools/tools-api/src/main/java/org/apache/kafka/tools/api/StringDecoder.java b/tools/tools-api/src/main/java/org/apache/kafka/tools/api/StringDecoder.java new file mode 100644 index 0000000000000..eb6b8ddff8a63 --- /dev/null +++ b/tools/tools-api/src/main/java/org/apache/kafka/tools/api/StringDecoder.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tools.api; + +import java.nio.charset.StandardCharsets; + +/** + * The string decoder translates bytes into strings. It uses UTF8 by default. + */ +public class StringDecoder implements Decoder { + @Override + public String fromBytes(byte[] bytes) { + return new String(bytes, StandardCharsets.UTF_8); + } +}